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 2021/09/28 14:17:19 UTC

[GitHub] [ignite-3] ibessonov opened a new pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

ibessonov opened a new pull request #365:
URL: https://github.com/apache/ignite-3/pull/365


   https://issues.apache.org/jira/browse/IGNITE-15351


-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723976144



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       I'll remove only those that are relevant to my code. Valut / Raft log / Metastorage are not related to partition storages.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";

Review comment:
       No, it's not possible. I'll extract constant for convenience

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {
+        if (CF_META.equals(cfName))
+            return CFType.META;
+
+        if (cfName.startsWith(CF_PARTITION_PREFIX))
+            return CFType.PARTITION;
+
+        if (cfName.startsWith(CF_INDEX_PREFIX))
+            return CFType.INDEX;
+
+        throw new StorageException("Unidentified column family [name=" + cfName + ", table=" + tableCfg.name() + ']');
+    }
+
+    /**
+     * Addes resource to the {@link #autoCloseables} list.

Review comment:
       Sorry

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       creating Stream in this context is just as complex as creating Iterator. I don't think I fully understand your reasoning right now. 




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723980830



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718411373



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       my failed attempt at code decomposition :(
   I'll make it static instead




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718409729



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/TableStorage.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.engine;
+
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+
+/**
+ * Table storage that contains meta, partitions and SQL indexes.
+ */
+public interface TableStorage {
+    /**
+     * Gets or creates a partition for current table.
+     *
+     * @param partId Partition id.
+     * @return Partition storage.
+     */
+    Storage getOrCreatePartition(int partId);

Review comment:
       Yes




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723982412



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718395042



##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/StorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.engine;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+
+/**
+ * General storageengine interface.
+ *
+ * @param <StorageCfg> Type of storage configuration.
+ * @param <RegionCfg> Type of data region configuration.
+ */
+public interface StorageEngine<
+    StorageCfg extends DataStorageConfiguration,

Review comment:
       I don't need them right now, they're for the future. I agree that they look out of place, I'll get rid of them




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718411618



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718390262



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       Contatenating streams is a lot more convenient then creating new collection or duplicate exception handling logic

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Nice!

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Nice! What's the difference?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));

Review comment:
       Done

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Done

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {

Review comment:
       That's a good point, hash map storage has nothing to do with production code

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       Technically, "createListener" and "getListenerPersistencePath" do not specify whether they will be called concurrently or not

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());
+                    ObjectInputStream objIn = new ObjectInputStream(in)
+                ) {
+                    var keys = (List<byte[]>)objIn.readObject();
+                    var values = (List<byte[]>)objIn.readObject();
+
+                    map.clear();
+
+                    for (int i = 0; i < keys.size(); i++)
+                        map.put(new ByteArray(keys.get(i)), values.get(i));
+                }
+                catch (Exception e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
+        });
+
+        paths.put(listener, tableDir);

Review comment:
       It's read in "getListenerPersistencePath", test won't work without it

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)

Review comment:
       Correct, these tests must be created in org.apache.ignite.internal.storage.AbstractStorageTest. I'll make them when I move CHMStorage into tests.

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/StorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.engine;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+
+/**
+ * General storageengine interface.
+ *
+ * @param <StorageCfg> Type of storage configuration.
+ * @param <RegionCfg> Type of data region configuration.
+ */
+public interface StorageEngine<
+    StorageCfg extends DataStorageConfiguration,

Review comment:
       I don't need them right now, they're for the future. I agree that they look out of place, I'll get rid of them

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */
+    private final Set<TableStorage> tableStorages = ConcurrentHashMap.newKeySet();

Review comment:
       I saw Kirill Tkalenko using it and was just as surprised, so shout out to him

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})

Review comment:
       This sucks :(

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})
+    @Value(hasDefault = true)
+    public String cache = "LRU";
+
+    /** Logarithm of the number of cache shards. {@code -1} if no shards required. */

Review comment:
       Good idea

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.StorageEngine;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.rocksdb.RocksDB;
+
+/**
+ * Storage engine implementation based on RocksDB.
+ */
+public class RocksDbStorageEngine implements StorageEngine<DataStorageConfiguration, DataRegionConfiguration> {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       For tests I guess, let me check if they would work without it

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory

Review comment:
       I wouldn't think too much about it right now because indexes are yet to be implemented.

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;

Review comment:
       This field and description will be moved to a proper schema in the future and I'll link it to actual WriteBufferManager.
   In short - it's all required for fine memory management

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       I can't, these are not Closeables

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -113,6 +121,9 @@
     /** Meta storage service. */
     private final MetaStorageManager metaStorageMgr;
 
+    /** Storage engine instance. Only one type is available right not, it's a {@link RocksDbStorageEngine}. */

Review comment:
       Done

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */

Review comment:
       Done

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->

Review comment:
       Totally agree with you

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->
             partitionsGroupsFutures.add(
                 raftMgr.prepareRaftGroup(
                     raftGroupName(tblId, p),
                     assignment.get(p),
                     () -> {
-                        Path storageDir = partitionsStoreDir.resolve(name);
-
-                        try {
-                            Files.createDirectories(storageDir);
-                        }
-                        catch (IOException e) {
-                            throw new IgniteInternalException(
-                                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
-                                e
-                            );
-                        }
+                        Storage storage = tableStorage.getOrCreatePartition(p);
 
-                        return new PartitionListener(
-                            new RocksDbStorage(
-                                storageDir.resolve(String.valueOf(p)),
-                                ByteBuffer::compareTo
-                            )
-                        );
+                        return new PartitionListener(storage);

Review comment:
       yes

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       It extends AbstractStorageTest which has quite a few tests for partition storage. This is a reason why I initialize everything in setUp - I have to initialize field "storage" before each test.

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       let me check that

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       I can't, these are not Closeables

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       let me check that. Ok, it works without extra casts.

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/TableStorage.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.engine;
+
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+
+/**
+ * Table storage that contains meta, partitions and SQL indexes.
+ */
+public interface TableStorage {
+    /**
+     * Gets or creates a partition for current table.
+     *
+     * @param partId Partition id.
+     * @return Partition storage.
+     */
+    Storage getOrCreatePartition(int partId);

Review comment:
       Yes

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       my failed attempt at code decomposition :(
   I'll make it static instead

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       Because list should be concurrent as well

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length(), cfName.length()));

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       It would be easier to replace it with Map right now, I'll do it

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))

Review comment:
       Done

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!

Review comment:
       I'll check why I thought this code is bad

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't see any problems with streams here, adding these collections to "copy" list would look bad because I'd have to prepend them, not append.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't see any problems with streams here, adding these collections to "copy" list would look bad because I'd have to prepend them, not append

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718392528



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {

Review comment:
       That's a good point, hash map storage has nothing to do with production code




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718464126



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       `File`-based API is considered deprecated, it's cleaner not to mix the new NIO `Path` API and the old one. 




-- 
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



[GitHub] [ignite-3] agura commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
agura commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723508272



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       @ibessonov Just remind you that we don't want to use Stream API in hot paths. I didn't like this method before your changes. Now I hate it :) Just because too many redundant transformations.  




-- 
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



[GitHub] [ignite-3] agura commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
agura commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r724001108



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       > 
   > 
   > > Here are some comments. Please, fix them.
   > > Also I have a question: what is default configuration for default region? As I remember, Ignite 2 starts with some defaults. But I cant find similar functionality in this change.
   > 
   > Defaults are described in org.apache.ignite.configuration.schemas.store.DataRegionConfigurationSchema class. They're not very generous right now, I used arbitrary values.
   
   Ok




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r717711965



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       wow, are these changes really necessary?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       you should use `Files.newOutputStream(snapshotPath.resolve("snapshot_file"))` instead

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));

Review comment:
       can be replaced with `new ArrayList<>(map.values())`

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       ```suggestion
       private final Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();
   ```

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})

Review comment:
       RocksDB documentation states that the Clock implementation has bugs and is not production-ready, maybe we shouldn't include it

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */
+    private final Set<TableStorage> tableStorages = ConcurrentHashMap.newKeySet();

Review comment:
       wow, I didn't know about the `newKeySet` method, that's awesome

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       please use `closeAll` instead

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());
+                    ObjectInputStream objIn = new ObjectInputStream(in)
+                ) {
+                    var keys = (List<byte[]>)objIn.readObject();
+                    var values = (List<byte[]>)objIn.readObject();
+
+                    map.clear();
+
+                    for (int i = 0; i < keys.size(); i++)
+                        map.put(new ByteArray(keys.get(i)), values.get(i));
+                }
+                catch (Exception e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
+        });
+
+        paths.put(listener, tableDir);

Review comment:
       I don't think this logic is needed: `tableDir` is not used anywhere, why create it at all?

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */

Review comment:
       ```suggestion
       /** Instances of table storages that need to be stopped on component stop. */
   ```

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       you can also write:
   ```
   IgniteUtils.closeAll(
       table == null ? null : table::stop,
       dataRegion == null ? null : dataRegion::stop
   );
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/TableStorage.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.engine;
+
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+
+/**
+ * Table storage that contains meta, partitions and SQL indexes.
+ */
+public interface TableStorage {
+    /**
+     * Gets or creates a partition for current table.
+     *
+     * @param partId Partition id.
+     * @return Partition storage.
+     */
+    Storage getOrCreatePartition(int partId);

Review comment:
       Shall we rename `Storage` to `PartitionStorage`?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {

Review comment:
       ```suggestion
       private static int partitionId(String cfName) {
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))

Review comment:
       please extract intermediate variables, it's currently hard to read

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {

Review comment:
       Can you split this method into smaller ones? For example, you can introduce a method called `getExistingCfDescriptors`

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length(), cfName.length()));

Review comment:
       ```suggestion
           return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
   ```

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       also, why is this a concurrent map? I can't find any concurrent access on it...

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.StorageEngine;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.rocksdb.RocksDB;
+
+/**
+ * Storage engine implementation based on RocksDB.
+ */
+public class RocksDbStorageEngine implements StorageEngine<DataStorageConfiguration, DataRegionConfiguration> {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       why do you need this here?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)

Review comment:
       Do I understand correctly that we no longer have tests for creating snapshots using the `RocksDbStorage`?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {

Review comment:
       why aren't these methods in the base class? Also, maybe we should move the `ConcurrentHashMapStorage` to a test package?

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})
+    @Value(hasDefault = true)
+    public String cache = "LRU";
+
+    /** Logarithm of the number of cache shards. {@code -1} if no shards required. */

Review comment:
       I would suggest to copy the documentation from RocksDb, current javadoc is hard to understand

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/StorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.engine;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+
+/**
+ * General storageengine interface.
+ *
+ * @param <StorageCfg> Type of storage configuration.
+ * @param <RegionCfg> Type of data region configuration.
+ */
+public interface StorageEngine<
+    StorageCfg extends DataStorageConfiguration,

Review comment:
       why do you need these generics?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       same here: `Files.newInputStream(snapshotPath.resolve("snapshot_file"))`

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;

Review comment:
       Why do we need the Write Buffer Manager here? Also, the property name is a bit misleading to me...

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -113,6 +121,9 @@
     /** Meta storage service. */
     private final MetaStorageManager metaStorageMgr;
 
+    /** Storage engine instance. Only one type is available right not, it's a {@link RocksDbStorageEngine}. */

Review comment:
       ```suggestion
       /** Storage engine instance. Only one type is available right now, which is the {@link RocksDbStorageEngine}. */
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory

Review comment:
       I would suggest creating a separate functional interface for this. Also the idea of index comparators is not clear to me...

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->
             partitionsGroupsFutures.add(
                 raftMgr.prepareRaftGroup(
                     raftGroupName(tblId, p),
                     assignment.get(p),
                     () -> {
-                        Path storageDir = partitionsStoreDir.resolve(name);
-
-                        try {
-                            Files.createDirectories(storageDir);
-                        }
-                        catch (IOException e) {
-                            throw new IgniteInternalException(
-                                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
-                                e
-                            );
-                        }
+                        Storage storage = tableStorage.getOrCreatePartition(p);
 
-                        return new PartitionListener(
-                            new RocksDbStorage(
-                                storageDir.resolve(String.valueOf(p)),
-                                ByteBuffer::compareTo
-                            )
-                        );
+                        return new PartitionListener(storage);

Review comment:
       Can we inline this stuff: `() -> new PartitionListener(tableStorage.getOrCreatePartition(p))` ?

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->

Review comment:
       I know that it is not related to this PR, but can you rewrite this code using a for loop? I love streams and all, but in this case they are a bit redundant

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       this function always accepts `tablePath` as an argument, can we inline that?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);

Review comment:
       ```suggestion
                   throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());

Review comment:
       can be written shorter: `partitionCfs = new ArrayList<>(Collections.nCopies(tableCfgView.partitions(), null));`

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       Why is this a `ConcurrentMap` while `partitionCf` is a regular list?

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       I think it would be easier to have a single test method instead of `setUp` and `tearDown`, it's quite confusing otherwise

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!

Review comment:
       ???

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       Also, this test is strange: what are you exactly trying to test here?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());

Review comment:
       or maybe it's even easier to use a regular array instead

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       you already create a copy of the `autoCloseables` list, I would suggest to add all other stuff there as well and get rid of this `concat` shenanigans

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       `File`-based API is considered deprecated, it's cleaner not to mix the new NIO `Path` API and the old one. 

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       Right, I haven't noticed the base class, sorry

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       I'm not talking about inlining the funciton, I'm talking about inlining the argument

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       you already create a copy of the `autoCloseables` list, I would suggest to add all other stuff there as well and get rid of these `concat` shenanigans

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't agree that you have to prepend them, since you call `reverse` at one place above, so you can append them beforehand. Also the code will look much simpler, if you also include the `IgniteUtils` changes




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723981161



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723979998



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723982221



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723980464



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718391714



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));

Review comment:
       Done

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718417506



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718414105



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length(), cfName.length()));

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718408310



##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       let me check that. Ok, it works without extra casts.




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723980282



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723976144



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       I'll remove only those that are relevant to my code. Valut / Raft log / Metastorage are not related to partition storages.




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718471622



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't agree that you have to prepend them, since you call `reverse` at one place above, so you can append them beforehand. Also the code will look much simpler, if you also include the `IgniteUtils` changes




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718404082



##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;

Review comment:
       This field and description will be moved to a proper schema in the future and I'll link it to actual WriteBufferManager.
   In short - it's all required for fine memory management




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718467143



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       I'm not talking about inlining the funciton, I'm talking about inlining the argument




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718404287



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       I can't, these are not Closeables

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718390262



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       Contatenating streams is a lot more convenient then creating new collection or duplicate exception handling logic




-- 
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



[GitHub] [ignite-3] agura commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
agura commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723508272



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       @ibessonov Just remind you that we don't want to use Stream API in hot paths. I didn't like this method before your changes. Now I hate it :) Just because the method does too many redundant transformations.  




-- 
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



[GitHub] [ignite-3] ibessonov merged pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov merged pull request #365:
URL: https://github.com/apache/ignite-3/pull/365


   


-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718405447



##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->

Review comment:
       Totally agree with you




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718408062



##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       It extends AbstractStorageTest which has quite a few tests for partition storage. This is a reason why I initialize everything in setUp - I have to initialize field "storage" before each test.




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718408310



##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       let me check that




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718412339



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       Because list should be concurrent as well




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r719199412



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {

Review comment:
       https://issues.apache.org/jira/browse/IGNITE-15646




-- 
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



[GitHub] [ignite-3] agura commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
agura commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r724001108



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       > 
   > 
   > > Here are some comments. Please, fix them.
   > > Also I have a question: what is default configuration for default region? As I remember, Ignite 2 starts with some defaults. But I cant find similar functionality in this change.
   > 
   > Defaults are described in org.apache.ignite.configuration.schemas.store.DataRegionConfigurationSchema class. They're not very generous right now, I used arbitrary values.
   
   Ok




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718394562



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)

Review comment:
       Correct, these tests must be created in org.apache.ignite.internal.storage.AbstractStorageTest. I'll make them when I move CHMStorage into tests.




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r717711965



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       wow, are these changes really necessary?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       you should use `Files.newOutputStream(snapshotPath.resolve("snapshot_file"))` instead

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));

Review comment:
       can be replaced with `new ArrayList<>(map.values())`

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       ```suggestion
       private final Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();
   ```

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})

Review comment:
       RocksDB documentation states that the Clock implementation has bugs and is not production-ready, maybe we shouldn't include it

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */
+    private final Set<TableStorage> tableStorages = ConcurrentHashMap.newKeySet();

Review comment:
       wow, I didn't know about the `newKeySet` method, that's awesome

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       please use `closeAll` instead

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());
+                    ObjectInputStream objIn = new ObjectInputStream(in)
+                ) {
+                    var keys = (List<byte[]>)objIn.readObject();
+                    var values = (List<byte[]>)objIn.readObject();
+
+                    map.clear();
+
+                    for (int i = 0; i < keys.size(); i++)
+                        map.put(new ByteArray(keys.get(i)), values.get(i));
+                }
+                catch (Exception e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
+        });
+
+        paths.put(listener, tableDir);

Review comment:
       I don't think this logic is needed: `tableDir` is not used anywhere, why create it at all?

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */

Review comment:
       ```suggestion
       /** Instances of table storages that need to be stopped on component stop. */
   ```

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(
+        @WorkDirectory Path workDir,
+        @InjectConfiguration DataRegionConfiguration dataRegionCfg,
+        @InjectConfiguration TableConfiguration tableCfg
+    ) throws Exception {
+        dataRegionCfg.change(cfg -> cfg.changeSize(16 * 1024).changeWriteBufferSize(16 * 1024)).get();
+
+        RocksDbStorageEngine engine = new RocksDbStorageEngine();
+
+        dataRegion = engine.createDataRegion(dataRegionCfg);
+
+        assertThat(dataRegion, is(instanceOf(RocksDbDataRegion.class)));
+
+        dataRegion.start();
+
+        table = engine.createTable(workDir, tableCfg, dataRegion, (tableView, indexName) -> null);
+
+        assertThat(table, is(instanceOf(RocksDbTableStorage.class)));
+
+        table.start();
+
+        storage = table.getOrCreatePartition(0);
+
+        assertThat(storage, is(instanceOf(RocksDbStorage.class)));
     }
 
     /** */
     @AfterEach
     public void tearDown() throws Exception {
-        if (storage != null)
-            ((AutoCloseable)storage).close();
+        try {

Review comment:
       you can also write:
   ```
   IgniteUtils.closeAll(
       table == null ? null : table::stop,
       dataRegion == null ? null : dataRegion::stop
   );
   ```

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/TableStorage.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.engine;
+
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+
+/**
+ * Table storage that contains meta, partitions and SQL indexes.
+ */
+public interface TableStorage {
+    /**
+     * Gets or creates a partition for current table.
+     *
+     * @param partId Partition id.
+     * @return Partition storage.
+     */
+    Storage getOrCreatePartition(int partId);

Review comment:
       Shall we rename `Storage` to `PartitionStorage`?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {

Review comment:
       ```suggestion
       private static int partitionId(String cfName) {
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))

Review comment:
       please extract intermediate variables, it's currently hard to read

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {

Review comment:
       Can you split this method into smaller ones? For example, you can introduce a method called `getExistingCfDescriptors`

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length(), cfName.length()));

Review comment:
       ```suggestion
           return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
   ```

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       also, why is this a concurrent map? I can't find any concurrent access on it...

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.StorageEngine;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.rocksdb.RocksDB;
+
+/**
+ * Storage engine implementation based on RocksDB.
+ */
+public class RocksDbStorageEngine implements StorageEngine<DataStorageConfiguration, DataRegionConfiguration> {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       why do you need this here?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)

Review comment:
       Do I understand correctly that we no longer have tests for creating snapshots using the `RocksDbStorage`?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {

Review comment:
       why aren't these methods in the base class? Also, maybe we should move the `ConcurrentHashMapStorage` to a test package?

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})
+    @Value(hasDefault = true)
+    public String cache = "LRU";
+
+    /** Logarithm of the number of cache shards. {@code -1} if no shards required. */

Review comment:
       I would suggest to copy the documentation from RocksDb, current javadoc is hard to understand

##########
File path: modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/StorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.engine;
+
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+
+/**
+ * General storageengine interface.
+ *
+ * @param <StorageCfg> Type of storage configuration.
+ * @param <RegionCfg> Type of data region configuration.
+ */
+public interface StorageEngine<
+    StorageCfg extends DataStorageConfiguration,

Review comment:
       why do you need these generics?

##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       same here: `Files.newInputStream(snapshotPath.resolve("snapshot_file"))`

##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;

Review comment:
       Why do we need the Write Buffer Manager here? Also, the property name is a bit misleading to me...

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -113,6 +121,9 @@
     /** Meta storage service. */
     private final MetaStorageManager metaStorageMgr;
 
+    /** Storage engine instance. Only one type is available right not, it's a {@link RocksDbStorageEngine}. */

Review comment:
       ```suggestion
       /** Storage engine instance. Only one type is available right now, which is the {@link RocksDbStorageEngine}. */
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory

Review comment:
       I would suggest creating a separate functional interface for this. Also the idea of index comparators is not clear to me...

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->
             partitionsGroupsFutures.add(
                 raftMgr.prepareRaftGroup(
                     raftGroupName(tblId, p),
                     assignment.get(p),
                     () -> {
-                        Path storageDir = partitionsStoreDir.resolve(name);
-
-                        try {
-                            Files.createDirectories(storageDir);
-                        }
-                        catch (IOException e) {
-                            throw new IgniteInternalException(
-                                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
-                                e
-                            );
-                        }
+                        Storage storage = tableStorage.getOrCreatePartition(p);
 
-                        return new PartitionListener(
-                            new RocksDbStorage(
-                                storageDir.resolve(String.valueOf(p)),
-                                ByteBuffer::compareTo
-                            )
-                        );
+                        return new PartitionListener(storage);

Review comment:
       Can we inline this stuff: `() -> new PartitionListener(tableStorage.getOrCreatePartition(p))` ?

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->

Review comment:
       I know that it is not related to this PR, but can you rewrite this code using a for loop? I love streams and all, but in this case they are a bit redundant

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       this function always accepts `tablePath` as an argument, can we inline that?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);

Review comment:
       ```suggestion
                   throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
   ```

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());

Review comment:
       can be written shorter: `partitionCfs = new ArrayList<>(Collections.nCopies(tableCfgView.partitions(), null));`

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       Why is this a `ConcurrentMap` while `partitionCf` is a regular list?

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       I think it would be easier to have a single test method instead of `setUp` and `tearDown`, it's quite confusing otherwise

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!

Review comment:
       ???

##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       Also, this test is strange: what are you exactly trying to test here?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());

Review comment:
       or maybe it's even easier to use a regular array instead

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       you already create a copy of the `autoCloseables` list, I would suggest to add all other stuff there as well and get rid of this `concat` shenanigans




-- 
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



[GitHub] [ignite-3] ibessonov commented on pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#issuecomment-937592535


   > Here are some comments. Please, fix them.
   > 
   > Also I have a question: what is default configuration for default region? As I remember, Ignite 2 starts with some defaults. But I cant find similar functionality in this change.
   
   Defaults are described in org.apache.ignite.configuration.schemas.store.DataRegionConfigurationSchema class. They're not very generous right now, I used arbitrary values.


-- 
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



[GitHub] [ignite-3] ibessonov merged pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov merged pull request #365:
URL: https://github.com/apache/ignite-3/pull/365


   


-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723992196



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
##########
@@ -467,27 +470,36 @@ public static void shutdownAndAwaitTermination(ExecutorService service, long tim
      * thrown exception will be propagated to the caller, after all other objects are closed, similar to
      * the try-with-resources block.
      *
-     * @param closeables Collection of objects to close.
+     * @param closeables Stream of objects to close.
      * @throws Exception If failed to close.
      */
-    public static void closeAll(Collection<? extends AutoCloseable> closeables) throws Exception {
-        Exception ex = null;
+    public static void closeAll(Stream<? extends AutoCloseable> closeables) throws Exception {

Review comment:
       creating Stream in this context is just as complex as creating Iterator. I don't think I fully understand your reasoning right now. 




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718402872



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory

Review comment:
       I wouldn't think too much about it right now because indexes are yet to be implemented.




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718401543



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageEngine.java
##########
@@ -0,0 +1,62 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.Comparator;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataStorageConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.StorageEngine;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.rocksdb.RocksDB;
+
+/**
+ * Storage engine implementation based on RocksDB.
+ */
+public class RocksDbStorageEngine implements StorageEngine<DataStorageConfiguration, DataRegionConfiguration> {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       For tests I guess, let me check if they would work without it




-- 
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



[GitHub] [ignite-3] agura commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
agura commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723516961



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();

Review comment:
       I see 7 usages of this method. Why should we invoke it from different places? It looks like a developer should guess where to place this invocation. I understand that result of the first correct invocation will be cached, but it just confuses.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());

Review comment:
       Why not a constant?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":

Review comment:
       Why not a constant?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {

Review comment:
       `Cf` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {

Review comment:
       `Cf` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":

Review comment:
       Why not a constant?

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,

Review comment:
       `column families`, not `columnfamilies`

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {

Review comment:
       `Cf` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {

Review comment:
       `Cf` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {

Review comment:
       `CF` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {

Review comment:
       `Cf` is too non-descriptive. Please, rename.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";

Review comment:
       Why is a comparator name hardcoded? Is it possible that this name will clash with other comparators? 

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {
+        if (CF_META.equals(cfName))
+            return CFType.META;
+
+        if (cfName.startsWith(CF_PARTITION_PREFIX))
+            return CFType.PARTITION;
+
+        if (cfName.startsWith(CF_INDEX_PREFIX))
+            return CFType.INDEX;
+
+        throw new StorageException("Unidentified column family [name=" + cfName + ", table=" + tableCfg.name() + ']');
+    }
+
+    /**
+     * Addes resource to the {@link #autoCloseables} list.

Review comment:
       Adds :)




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718416910



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, path);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + path,
+                e
+            );
+        }
+
+        return cfNames;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private int partitionId(String cfName) {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718404653



##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -113,6 +121,9 @@
     /** Meta storage service. */
     private final MetaStorageManager metaStorageMgr;
 
+    /** Storage engine instance. Only one type is available right not, it's a {@link RocksDbStorageEngine}. */

Review comment:
       Done

##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718417697



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!

Review comment:
       I'll check why I thought this code is bad




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718468704



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r719199628



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)

Review comment:
       https://issues.apache.org/jira/browse/IGNITE-15646




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r719202087



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());

Review comment:
       I think I'll use AtomicReferenceArray




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718399179



##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})

Review comment:
       This sucks :(




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718404287



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.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.rocksdb;
+
+import java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() {
+        DataRegionView dataRegionView = cfg.value();
+
+        long writeBufferSize = dataRegionView.writeBufferSize();
+
+        long totalCacheSize = dataRegionView.size() + writeBufferSize;
+
+        switch (dataRegionView.cache().toLowerCase(Locale.ROOT)) {
+            case "clock":
+                cache = new ClockCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            case "lru":
+                cache = new LRUCache(totalCacheSize, dataRegionView.numShardBits(), false);
+
+                break;
+
+            default:
+                assert false : dataRegionView.cache();
+        }
+
+        writeBufferManager = new WriteBufferManager(writeBufferSize, cache);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() {
+        if (writeBufferManager != null)

Review comment:
       I can't, these are not Closeables




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718390425



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Nice! What's the difference?




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718396070



##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -122,27 +133,39 @@
     /** Tables. */
     private final Map<IgniteUuid, TableImpl> tablesById = new ConcurrentHashMap<>();
 
+    /** Default data region instance. */
+    private DataRegion defaultDataRegion;
+
+    //TODO: IGNITE-15161 These should go into TableImpl instances.
+    /** Instances of table storages that need to be stop on component stop. */
+    private final Set<TableStorage> tableStorages = ConcurrentHashMap.newKeySet();

Review comment:
       I saw Kirill Tkalenko using it and was just as surprised, so shout out to him




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718385467



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       you already create a copy of the `autoCloseables` list, I would suggest to add all other stuff there as well and get rid of these `concat` shenanigans




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718406863



##########
File path: modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
##########
@@ -253,30 +295,40 @@ private void createTableLocally(
 
         var partitionsGroupsFutures = new ArrayList<CompletableFuture<RaftGroupService>>();
 
+        Path storageDir = partitionsStoreDir.resolve(name);
+
+        try {
+            Files.createDirectories(storageDir);
+        }
+        catch (IOException e) {
+            throw new IgniteInternalException(
+                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
+                e
+            );
+        }
+
+        TableStorage tableStorage = engine.createTable(
+            storageDir,
+            tablesCfg.tables().get(name),
+            defaultDataRegion,
+            (tableCfgView, indexName) -> {
+                throw new UnsupportedOperationException("Not implemented yet.");
+            }
+        );
+
+        tableStorage.start();
+
+        tableStorages.add(tableStorage);
+
         IntStream.range(0, partitions).forEach(p ->
             partitionsGroupsFutures.add(
                 raftMgr.prepareRaftGroup(
                     raftGroupName(tblId, p),
                     assignment.get(p),
                     () -> {
-                        Path storageDir = partitionsStoreDir.resolve(name);
-
-                        try {
-                            Files.createDirectories(storageDir);
-                        }
-                        catch (IOException e) {
-                            throw new IgniteInternalException(
-                                "Failed to create partitions store directory for " + name + ": " + e.getMessage(),
-                                e
-                            );
-                        }
+                        Storage storage = tableStorage.getOrCreatePartition(p);
 
-                        return new PartitionListener(
-                            new RocksDbStorage(
-                                storageDir.resolve(String.valueOf(p)),
-                                ByteBuffer::compareTo
-                            )
-                        );
+                        return new PartitionListener(storage);

Review comment:
       yes




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718390425



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());

Review comment:
       Nice!




-- 
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



[GitHub] [ignite-3] sashapolo commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
sashapolo commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718465803



##########
File path: modules/storage-rocksdb/src/test/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageTest.java
##########
@@ -17,30 +17,74 @@
 
 package org.apache.ignite.internal.storage.rocksdb;
 
-import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
+import org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
 import org.apache.ignite.internal.storage.AbstractStorageTest;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.testframework.WorkDirectory;
 import org.apache.ignite.internal.testframework.WorkDirectoryExtension;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.extension.ExtendWith;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
 /**
  * Storage test implementation for {@link RocksDbStorage}.
  */
 @ExtendWith(WorkDirectoryExtension.class)
+@ExtendWith(ConfigurationExtension.class)
 public class RocksDbStorageTest extends AbstractStorageTest {
+    /** */
+    private TableStorage table;
+
+    /** */
+    private DataRegion dataRegion;
+
     /** */
     @BeforeEach
-    public void setUp(@WorkDirectory Path workDir) {
-        storage = new RocksDbStorage(workDir, ByteBuffer::compareTo);
+    public void setUp(

Review comment:
       Right, I haven't noticed the base class, sorry




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723978137



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbDataRegion.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 java.util.Locale;
+import org.apache.ignite.configuration.schemas.store.DataRegionConfiguration;
+import org.apache.ignite.configuration.schemas.store.DataRegionView;
+import org.apache.ignite.internal.storage.engine.DataRegion;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.rocksdb.Cache;
+import org.rocksdb.ClockCache;
+import org.rocksdb.LRUCache;
+import org.rocksdb.RocksDB;
+import org.rocksdb.WriteBufferManager;
+
+/**
+ * Data region implementation for {@link RocksDbStorageEngine}. Based on a {@link Cache}.
+ */
+public class RocksDbDataRegion implements DataRegion {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /** Region configuration. */
+    private final DataRegionConfiguration cfg;
+
+    /** RocksDB cache instance. */
+    private Cache cache;
+
+    /** Write buffer manager instance. */
+    private WriteBufferManager writeBufferManager;
+
+    /**
+     * Constructor.
+     *
+     * @param cfg Data region configuration.
+     */
+    public RocksDbDataRegion(DataRegionConfiguration cfg) {
+        this.cfg = cfg;
+
+        assert "rocksdb".equalsIgnoreCase(cfg.type().value());

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r719207033



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Storage getOrCreatePartition(int partId) {
+        assert partId < partitionCfs.size() : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitionCfs.size(), false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Feiled to create new ROcksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path.
+     *
+     * @param path DB path.
+     * @return List of column families names.
+     * @throws StorageException If something went wrong.
+     */
+    @NotNull private List<String> listCfNames(String path) throws StorageException {

Review comment:
       ok




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723982868



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";

Review comment:
       No, it's not possible. I'll extract constant for convenience




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723981670



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {

Review comment:
       Done




-- 
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



[GitHub] [ignite-3] ibessonov commented on pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#issuecomment-937592535


   > Here are some comments. Please, fix them.
   > 
   > Also I have a question: what is default configuration for default region? As I remember, Ignite 2 starts with some defaults. But I cant find similar functionality in this change.
   
   Defaults are described in org.apache.ignite.configuration.schemas.store.DataRegionConfigurationSchema class. They're not very generous right now, I used arbitrary values.


-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718393399



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -63,6 +78,9 @@
     /** */
     private static final Row SECOND_VALUE = createKeyValueRow(1, 1);
 
+    /** Paths for created partition listeners. */
+    private Map<PartitionListener, Path> paths = new ConcurrentHashMap<>();

Review comment:
       Technically, "createListener" and "getListenerPersistencePath" do not specify whether they will be called concurrently or not




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718414416



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();

Review comment:
       It would be easier to replace it with Map right now, I'll do it




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r723983575



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,382 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.function.BiFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.PartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as an array for the quick access by an index. */
+    private AtomicReferenceArray<ColumnFamily> partitionCfs;
+
+    /** Max number of partitions in the table. */
+    private int partitions;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        Map<CFType, List<String>> cfNamesGrouped = getCfNames();
+
+        List<ColumnFamilyDescriptor> cfDescriptors = convertToCfDescriptors(cfNamesGrouped);
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitions = tableCfg.value().partitions();
+
+        partitionCfs = new AtomicReferenceArray<>(partitions);
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(IntStream.range(0, partitions).mapToObj(partitionCfs::get), indicesCfHandles.values().stream()),
+                copy.stream()
+            ));
+        }
+        catch (Exception e) {
+            throw new StorageException("Failed to stop RocksDB table storage.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public PartitionStorage getOrCreatePartition(int partId) {
+        assert partId < partitions : S.toString(
+            "Attempt to create partition with id outside of configured range",
+            "partitionId", partId, false,
+            "partitions", partitions, false
+        );
+
+        ColumnFamily partitionCf = partitionCfs.get(partId);
+
+        if (partitionCf == null) {
+            String handleName = partitionCfName(partId);
+
+            ColumnFamilyDescriptor cfDescriptor = new ColumnFamilyDescriptor(
+                handleName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            );
+
+            try {
+                ColumnFamilyHandle cfHandle = db.createColumnFamily(cfDescriptor);
+
+                partitionCf = new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null);
+            }
+            catch (RocksDBException e) {
+                cfDescriptor.getOptions().close();
+
+                throw new StorageException("Failed to create new RocksDB column family " + handleName, e);
+            }
+
+            partitionCfs.set(partId, partitionCf);
+        }
+
+        return new RocksDbPartitionStorage(db, partitionCf);
+    }
+
+    /**
+     * Returns list of column families names that belong to RocksDB instance in the given path, grouped by thier
+     * {@link CFType}.
+     *
+     * @return Map with column families names.
+     * @throws StorageException If something went wrong.
+     */
+    private Map<CFType, List<String>> getCfNames() {
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = new ArrayList<>();
+
+        try (Options opts = new Options()) {
+            List<byte[]> cfNamesBytes = RocksDB.listColumnFamilies(opts, absolutePathStr);
+
+            for (byte[] cfNameBytes : cfNamesBytes)
+                cfNames.add(new String(cfNameBytes, StandardCharsets.UTF_8));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException(
+                "Failed to read list of columnfamilies names for the RocksDB instance located at path " + absolutePathStr,
+                e
+            );
+        }
+
+        return cfNames.stream().collect(groupingBy(this::cfType));
+    }
+
+    /**
+     * Returns list of CF descriptors by their names.
+     *
+     * @param cfGrouped Map from CF type to lists of names.
+     * @return List of CF descriptors.
+     */
+    @NotNull private List<ColumnFamilyDescriptor> convertToCfDescriptors(Map<CFType, List<String>> cfGrouped) {
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        Options cfOptions = addToCloseableResources(new Options().setCreateIfMissing(true));
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(cfOptions))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of())) {
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+        }
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfg.value().indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : "Found index that is absent in configuration: " + indexCfName;
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfg.value(), indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        return cfDescriptors;
+    }
+
+    /**
+     * Creates column family name by partition id.
+     *
+     * @param partId Partition id.
+     * @return Column family name.
+     */
+    private static String partitionCfName(int partId) {
+        return CF_PARTITION_PREFIX + partId;
+    }
+
+    /**
+     * Gets partition id from column family name.
+     *
+     * @param cfName Column family name.
+     * @return Partition id.
+     */
+    private static int partitionId(String cfName) {
+        return parseInt(cfName.substring(CF_PARTITION_PREFIX.length()));
+    }
+
+    /**
+     * Creates column family name by index name.
+     *
+     * @param idxName Index name.
+     * @return Column family name.
+     */
+    private static String indexCfName(String idxName) {
+        return CF_INDEX_PREFIX + idxName;
+    }
+
+    /**
+     * Determines column family type by its name.
+     *
+     * @param cfName Column family name.
+     * @return Column family type.
+     * @throws StorageException If column family name doesn't match any known pattern.
+     */
+    private CFType cfType(String cfName) throws StorageException {
+        if (CF_META.equals(cfName))
+            return CFType.META;
+
+        if (cfName.startsWith(CF_PARTITION_PREFIX))
+            return CFType.PARTITION;
+
+        if (cfName.startsWith(CF_INDEX_PREFIX))
+            return CFType.INDEX;
+
+        throw new StorageException("Unidentified column family [name=" + cfName + ", table=" + tableCfg.name() + ']');
+    }
+
+    /**
+     * Addes resource to the {@link #autoCloseables} list.

Review comment:
       Sorry




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718400821



##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/store/DataRegionConfigurationSchema.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.configuration.schemas.store;
+
+import org.apache.ignite.configuration.annotation.Config;
+import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Min;
+import org.apache.ignite.configuration.validation.OneOf;
+
+/**
+ * Configuration schema for data region. Currently it represents configuration for rocksdb storage engine only.
+ */
+@Config
+public class DataRegionConfigurationSchema {
+    /** Type for the future polymorphic configuration schemas. */
+    @OneOf("rocksdb")
+    @Value(hasDefault = true)
+    public String type = "rocksdb";
+
+    /** Size of the rocksdb offheap cache. */
+    @Value(hasDefault = true)
+    public int size = 64 * 1024 * 1024;
+
+    /** Size of rocksdb write buffer. */
+    @Value(hasDefault = true)
+    @Min(1)
+    public long writeBufferSize = 64 * 1024 * 1024;
+
+    /** Cache type - either {@code Clock} or {@code LRU}. */
+    @OneOf({"Clock", "LRU"})
+    @Value(hasDefault = true)
+    public String cache = "LRU";
+
+    /** Logarithm of the number of cache shards. {@code -1} if no shards required. */

Review comment:
       Good idea




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718468419



##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't see any problems with streams here, adding these collections to "copy" list would look bad because I'd have to prepend them, not append.

##########
File path: modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
##########
@@ -0,0 +1,366 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.BiFunction;
+import org.apache.ignite.configuration.NamedListView;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.internal.rocksdb.ColumnFamily;
+import org.apache.ignite.internal.storage.Storage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.engine.TableStorage;
+import org.apache.ignite.internal.tostring.S;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.jetbrains.annotations.NotNull;
+import org.rocksdb.AbstractComparator;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ColumnFamilyOptions;
+import org.rocksdb.ComparatorOptions;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Options;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Stream.concat;
+
+/**
+ * Table storage implementation based on {@link RocksDB} instance.
+ */
+public class RocksDbTableStorage implements TableStorage {
+    static {
+        RocksDB.loadLibrary();
+    }
+
+    /**
+     * Name of the meta column family matches default columns family, meaning that it always exist when new table is
+     * created.
+     */
+    private static final String CF_META = "default";
+
+    /** Prefix for partitions column families names. */
+    private static final String CF_PARTITION_PREFIX = "cf-part:";
+
+    /** Prefix for SQL indexes column family names. */
+    private static final String CF_INDEX_PREFIX = "cf-idx:";
+
+    /** Path for the directory that stores table data. */
+    private final Path tablePath;
+
+    /** Table configuration. */
+    private final TableConfiguration tableCfg;
+
+    /** Data region for the table. */
+    private final RocksDbDataRegion dataRegion;
+
+    /** Comparators factory for indexes. */
+    private final BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory;
+
+    /** List of closeable resources to close on {@link #stop()}. Better than having a field for each one of them. */
+    private final List<AutoCloseable> autoCloseables = new ArrayList<>();
+
+    /** Rocks DB instance itself. */
+    private RocksDB db;
+
+    /** CF handle for meta information. */
+    @SuppressWarnings("unused")
+    private ColumnFamilyHandle metaCfHandle;
+
+    /** Column families for partitions. Stored as a list for the quick access by an index. */
+    private List<ColumnFamily> partitionCfs;
+
+    /** Column families for indexes by their names. */
+    private Map<String, ColumnFamilyHandle> indicesCfHandles = new ConcurrentHashMap<>();
+
+    /** Utility enum to describe a type of the column family - meta, partition or index. */
+    private enum CFType {
+        META, PARTITION, INDEX
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param tablePath Path for the directory that stores table data.
+     * @param tableCfg Table configuration.
+     * @param dataRegion Data region for the table.
+     * @param indexComparatorFactory Comparators factory for indexes.
+     */
+    public RocksDbTableStorage(
+        Path tablePath,
+        TableConfiguration tableCfg,
+        RocksDbDataRegion dataRegion,
+        BiFunction<TableView, String, Comparator<ByteBuffer>> indexComparatorFactory
+    ) {
+        this.tablePath = tablePath;
+        this.tableCfg = tableCfg;
+        this.dataRegion = dataRegion;
+        this.indexComparatorFactory = indexComparatorFactory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws StorageException {
+        DBOptions dbOptions = addToCloseableResources(new DBOptions()
+            .setCreateIfMissing(true)
+            .setWriteBufferManager(dataRegion.writeBufferManager())
+        );
+
+        String absolutePathStr = tablePath.toAbsolutePath().toString();
+
+        List<String> cfNames = listCfNames(absolutePathStr);
+
+        Map<CFType, List<String>> cfGrouped = cfNames.stream().collect(groupingBy(this::cfType));
+
+        List<ColumnFamilyDescriptor> cfDescriptors = new ArrayList<>();
+
+        cfDescriptors.add(new ColumnFamilyDescriptor(
+            CF_META.getBytes(StandardCharsets.UTF_8),
+            addToCloseableResources(new ColumnFamilyOptions(addToCloseableResources(new Options().setCreateIfMissing(true))))
+        ));
+
+        for (String partitionCfName : cfGrouped.getOrDefault(CFType.PARTITION, List.of()))
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                partitionCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+            ));
+
+        TableView tableCfgView = tableCfg.value();
+
+        NamedListView<? extends TableIndexView> indicesCfgView = tableCfgView.indices();
+
+        for (String indexCfName : cfGrouped.getOrDefault(CFType.INDEX, List.of())) {
+            String indexName = indexCfName.substring(CF_INDEX_PREFIX.length());
+
+            TableIndexView indexCfgView = indicesCfgView.get(indexName);
+
+            assert indexCfgView != null : indexCfName; //TODO Bullshit!
+
+            Comparator<ByteBuffer> indexComparator = indexComparatorFactory.apply(tableCfgView, indexName);
+
+            cfDescriptors.add(new ColumnFamilyDescriptor(
+                indexCfName.getBytes(StandardCharsets.UTF_8),
+                new ColumnFamilyOptions()
+                    .setComparator(addToCloseableResources(
+                        new AbstractComparator(addToCloseableResources(new ComparatorOptions())) {
+                            /** {@inheritDoc} */
+                            @Override public String name() {
+                                return "index-comparator";
+                            }
+
+                            /** {@inheritDoc} */
+                            @Override public int compare(ByteBuffer a, ByteBuffer b) {
+                                return indexComparator.compare(a, b);
+                            }
+                        }))
+            ));
+        }
+
+        List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+
+        try {
+            db = addToCloseableResources(RocksDB.open(dbOptions, absolutePathStr, cfDescriptors, cfHandles));
+        }
+        catch (RocksDBException e) {
+            throw new StorageException("Failed to initialize RocksDB instance.", e);
+        }
+
+        partitionCfs = new ArrayList<>(tableCfgView.partitions());
+
+        partitionCfs.addAll(Collections.nCopies(tableCfgView.partitions(), null));
+
+        for (int cfListIndex = 0; cfListIndex < cfHandles.size(); cfListIndex++) {
+            ColumnFamilyHandle cfHandle = cfHandles.get(cfListIndex);
+
+            String handleName;
+            try {
+                handleName = new String(cfHandle.getName(), StandardCharsets.UTF_8);
+            }
+            catch (RocksDBException e) {
+                throw new StorageException("Failed to read RocksDB column family name.", e);
+            }
+
+            if (handleName.equals(CF_META))
+                this.metaCfHandle = addToCloseableResources(cfHandle);
+            else if (handleName.startsWith(CF_PARTITION_PREFIX)) {
+                int partId = partitionId(handleName);
+
+                ColumnFamilyDescriptor cfDescriptor = cfDescriptors.get(cfListIndex);
+
+                partitionCfs.set(partId, new ColumnFamily(db, cfHandle, handleName, cfDescriptor.getOptions(), null));
+            }
+            else {
+                String indexName = handleName.substring(CF_INDEX_PREFIX.length());
+
+                indicesCfHandles.put(indexName, cfHandle);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws StorageException {
+        try {
+            List<AutoCloseable> copy = new ArrayList<>(autoCloseables);
+
+            Collections.reverse(copy);
+
+            IgniteUtils.closeAll(concat(
+                concat(partitionCfs.stream(), indicesCfHandles.values().stream()),

Review comment:
       I don't see any problems with streams here, adding these collections to "copy" list would look bad because I'd have to prepend them, not append




-- 
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



[GitHub] [ignite-3] ibessonov commented on a change in pull request #365: IGNITE-15351 Implemented concepts of storage engines and data regions with basic integration into existing code.

Posted by GitBox <gi...@apache.org>.
ibessonov commented on a change in pull request #365:
URL: https://github.com/apache/ignite-3/pull/365#discussion_r718393804



##########
File path: modules/table/src/integrationTest/java/org/apache/ignite/distributed/ITTablePersistenceTest.java
##########
@@ -125,14 +143,53 @@
 
     /** {@inheritDoc} */
     @Override public Path getListenerPersistencePath(PartitionListener listener) {
-        return ((RocksDbStorage) listener.getStorage()).getDbPath();
+        return paths.get(listener);
     }
 
     /** {@inheritDoc} */
     @Override public RaftGroupListener createListener(Path workDir) {
-        return new PartitionListener(
-            new RocksDbStorage(workDir.resolve(UUID.randomUUID().toString()), ByteBuffer::compareTo)
-        );
+        Path tableDir = workDir.resolve(UUID.randomUUID().toString());
+
+        PartitionListener listener = new PartitionListener(new ConcurrentHashMapStorage() {
+            /** {@inheritDoc} */
+            @Override public @NotNull CompletableFuture<Void> snapshot(Path snapshotPath) {
+                return CompletableFuture.runAsync(() -> {
+                    try (
+                        OutputStream out = new FileOutputStream(snapshotPath.resolve("snapshot_file").toFile());
+                        ObjectOutputStream objOut = new ObjectOutputStream(out)
+                    ) {
+                        objOut.writeObject(map.keySet().stream().map(ByteArray::bytes).collect(toList()));
+                        objOut.writeObject(map.values().stream().collect(toList()));
+                    }
+                    catch (Exception e) {
+                        throw new IgniteInternalException(e);
+                    }
+                });
+            }
+
+            /** {@inheritDoc} */
+            @Override public void restoreSnapshot(Path snapshotPath) {
+                try (
+                    InputStream in = new FileInputStream(snapshotPath.resolve("snapshot_file").toFile());
+                    ObjectInputStream objIn = new ObjectInputStream(in)
+                ) {
+                    var keys = (List<byte[]>)objIn.readObject();
+                    var values = (List<byte[]>)objIn.readObject();
+
+                    map.clear();
+
+                    for (int i = 0; i < keys.size(); i++)
+                        map.put(new ByteArray(keys.get(i)), values.get(i));
+                }
+                catch (Exception e) {
+                    throw new IgniteInternalException(e);
+                }
+            }
+        });
+
+        paths.put(listener, tableDir);

Review comment:
       It's read in "getListenerPersistencePath", test won't work without it




-- 
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