You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ni...@apache.org on 2022/06/30 09:07:57 UTC

[ignite] branch master updated: IGNITE-17236 Inline statistics added to index-reader (#10116)

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

nizhikov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 1d6c47fce28 IGNITE-17236 Inline statistics added to index-reader (#10116)
1d6c47fce28 is described below

commit 1d6c47fce28342ba359a1cb0dce2c5a65e47dec4
Author: Nikolay <ni...@apache.org>
AuthorDate: Thu Jun 30 12:07:51 2022 +0300

    IGNITE-17236 Inline statistics added to index-reader (#10116)
---
 .../commandline/indexreader/IgniteIndexReader.java | 287 +++++++++++++++++++--
 .../IgniteIndexReaderFilePageStoreFactory.java     | 133 ----------
 .../commandline/indexreader/ScanContext.java       |  13 +-
 .../indexreader/IgniteIndexReaderTest.java         |  35 +--
 .../inline/types/NullableInlineIndexKeyType.java   |   2 +-
 5 files changed, 293 insertions(+), 177 deletions(-)

diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReader.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReader.java
index b691972484e..97432447e42 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReader.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReader.java
@@ -18,7 +18,10 @@
 package org.apache.ignite.internal.commandline.indexreader;
 
 import java.io.File;
+import java.io.IOException;
+import java.io.ObjectInputStream;
 import java.nio.ByteBuffer;
+import java.nio.file.Files;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -33,6 +36,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.function.Predicate;
 import java.util.function.Supplier;
+import java.util.logging.Level;
 import java.util.logging.Logger;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -40,19 +44,31 @@ import java.util.stream.IntStream;
 import java.util.stream.LongStream;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.internal.cache.query.index.IndexProcessor;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings;
+import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypes;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyType;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.InlineIndexKeyTypeRegistry;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.io.AbstractInlineLeafIO;
 import org.apache.ignite.internal.cache.query.index.sorted.inline.io.InlineIO;
+import org.apache.ignite.internal.cache.query.index.sorted.inline.types.NullableInlineIndexKeyType;
 import org.apache.ignite.internal.commandline.CommandHandler;
 import org.apache.ignite.internal.commandline.ProgressPrinter;
 import org.apache.ignite.internal.commandline.argument.parser.CLIArgumentParser;
 import org.apache.ignite.internal.commandline.indexreader.ScanContext.PagesStatistic;
 import org.apache.ignite.internal.commandline.systemview.SystemViewCommand;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
+import org.apache.ignite.internal.pagemem.PageUtils;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
 import org.apache.ignite.internal.processors.cache.persistence.IndexStorageImpl;
 import org.apache.ignite.internal.processors.cache.persistence.StorageException;
+import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
 import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreV2;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileVersionCheckingFactory;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.io.PagesListMetaIO;
 import org.apache.ignite.internal.processors.cache.persistence.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.processors.cache.persistence.tree.io.AbstractDataPageIO;
@@ -69,12 +85,15 @@ import org.apache.ignite.internal.processors.cache.persistence.wal.crc.IgniteDat
 import org.apache.ignite.internal.processors.cache.tree.AbstractDataLeafIO;
 import org.apache.ignite.internal.processors.cache.tree.PendingRowIO;
 import org.apache.ignite.internal.processors.cache.tree.RowLinkIO;
+import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.lang.GridPlainClosure2;
+import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.lang.RunnableX;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
@@ -85,6 +104,7 @@ import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
 import static java.util.Objects.isNull;
 import static java.util.Objects.nonNull;
+import static java.util.logging.Level.WARNING;
 import static java.util.stream.Collectors.joining;
 import static java.util.stream.Collectors.toList;
 import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_PAGE_SIZE;
@@ -98,7 +118,9 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId;
 import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId;
 import static org.apache.ignite.internal.pagemem.PageIdUtils.pageIndex;
 import static org.apache.ignite.internal.pagemem.PageIdUtils.partId;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME;
 import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
 import static org.apache.ignite.internal.util.GridUnsafe.allocateBuffer;
 import static org.apache.ignite.internal.util.GridUnsafe.bufferAddress;
 import static org.apache.ignite.internal.util.GridUnsafe.freeBuffer;
@@ -142,6 +164,10 @@ public class IgniteIndexReader implements AutoCloseable {
     /** */
     private static final String CHECK_PARTS_ARG = "--check-parts";
 
+    /** */
+    private static final Pattern CACHE_TYPE_ID_INDEX_SEARCH_PATTERN =
+        Pattern.compile("(?<id>[-0-9]{1,15})_(?<typeId>[-0-9]{1,15})_(?<indexName>.*)##.*");
+
     /** */
     private static final Pattern CACHE_TYPE_ID_SEARCH_PATTERN =
         Pattern.compile("(?<id>[-0-9]{1,15})_(?<typeId>[-0-9]{1,15})_.*");
@@ -153,10 +179,16 @@ public class IgniteIndexReader implements AutoCloseable {
     /** */
     private static final int MAX_ERRORS_CNT = 10;
 
+    /** */
+    static final int UNKNOWN_CACHE = -1;
+
     static {
         IndexProcessor.registerIO();
     }
 
+    /** Directory with data(partitions and index). */
+    private final File root;
+
     /** Page size. */
     private final int pageSize;
 
@@ -178,6 +210,9 @@ public class IgniteIndexReader implements AutoCloseable {
     /** Partitions page stores, may contains {@code null}. */
     private final FilePageStore[] partStores;
 
+    /** */
+    private final Map<Integer, StoredCacheData> storedCacheData = new HashMap<>();
+
     /** */
     private final Set<Integer> missingPartitions = new HashSet<>();
 
@@ -197,29 +232,48 @@ public class IgniteIndexReader implements AutoCloseable {
     private final LevelsPageVisitor levelsPageVisitor = new LevelsPageVisitor();
 
     /** */
-    private final Map<String, IgnitePair<Integer>> cacheTypeIds = new HashMap<>();
+    private final Map<String, GridTuple3<Integer, Integer, String>> cacheTypeIds = new HashMap<>();
 
     /**
      * Constructor.
      *
-     * @param idxFilter Index name filter, if {@code null} then is not used.
+     * @param pageSize Page size.
+     * @param partCnt Page count.
+     * @param filePageStoreVer Version of file page store.
      * @param checkParts Check cache data tree in partition files and it's consistency with indexes.
-     * @param filePageStoreFactory File page store factory.
+     * @param root Root directory.
+     * @param idxFilter Index name filter, if {@code null} then is not used.
      * @param log Logger.
      * @throws IgniteCheckedException If failed.
      */
     public IgniteIndexReader(
+        int pageSize,
+        int partCnt,
+        int filePageStoreVer,
+        File root,
         @Nullable Predicate<String> idxFilter,
         boolean checkParts,
-        IgniteIndexReaderFilePageStoreFactory filePageStoreFactory,
         Logger log
     ) throws IgniteCheckedException {
-        pageSize = filePageStoreFactory.pageSize();
-        partCnt = filePageStoreFactory.partitionCount();
+        this.pageSize = pageSize;
+        this.partCnt = partCnt;
+        this.root = root;
         this.checkParts = checkParts;
         this.idxFilter = idxFilter;
         this.log = log;
-        idxStore = filePageStoreFactory.createFilePageStore(INDEX_PARTITION, FLAG_IDX);
+
+        FileVersionCheckingFactory storeFactory = new FileVersionCheckingFactory(
+            new AsyncFileIOFactory(),
+            new AsyncFileIOFactory(),
+            () -> pageSize
+        ) {
+            /** {@inheritDoc} */
+            @Override public int latestVersion() {
+                return filePageStoreVer;
+            }
+        };
+
+        idxStore = filePageStore(INDEX_PARTITION, FLAG_IDX, storeFactory);
 
         if (isNull(idxStore))
             throw new IgniteCheckedException(INDEX_FILE_NAME + " file not found");
@@ -229,7 +283,18 @@ public class IgniteIndexReader implements AutoCloseable {
         partStores = new FilePageStore[partCnt];
 
         for (int i = 0; i < partCnt; i++)
-            partStores[i] = filePageStoreFactory.createFilePageStore(i, FLAG_DATA);
+            partStores[i] = filePageStore(i, FLAG_DATA, storeFactory);
+
+        Arrays.stream(root.listFiles(f -> f.getName().endsWith(CACHE_DATA_FILENAME))).forEach(f -> {
+            try (ObjectInputStream stream = new ObjectInputStream(Files.newInputStream(f.toPath()))) {
+                StoredCacheData data = (StoredCacheData)stream.readObject();
+
+                storedCacheData.put(CU.cacheId(data.config().getName()), data);
+            }
+            catch (ClassNotFoundException | IOException e) {
+                log.log(WARNING, "Can't read stored cache data. Inline for this cache will not be analyzed [f=" + f.getName() + ']', e);
+            }
+        });
     }
 
     /**
@@ -263,19 +328,15 @@ public class IgniteIndexReader implements AutoCloseable {
 
         p.parse(asList(args).iterator());
 
-        IgniteIndexReaderFilePageStoreFactory filePageStoreFactory = new IgniteIndexReaderFilePageStoreFactory(
-            new File(p.<String>get(DIR_ARG)),
-            p.get(PAGE_SIZE_ARG),
-            p.get(PART_CNT_ARG),
-            p.get(PAGE_STORE_VER_ARG)
-        );
-
         Set<String> idxs = new HashSet<>(asList(p.get(INDEXES_ARG)));
 
         try (IgniteIndexReader reader = new IgniteIndexReader(
+            p.get(PAGE_SIZE_ARG),
+            p.get(PART_CNT_ARG),
+            p.get(PAGE_STORE_VER_ARG),
+            new File(p.<String>get(DIR_ARG)),
             idxs.isEmpty() ? null : idxs::contains,
             p.get(CHECK_PARTS_ARG),
-            filePageStoreFactory,
             CommandHandler.setupJavaLogger("index-reader", IgniteIndexReader.class)
         )) {
             reader.readIndex();
@@ -358,7 +419,7 @@ public class IgniteIndexReader implements AutoCloseable {
      * @return Tree traversal context.
      */
     ScanContext recursiveTreeScan(long rootPageId, String idx, ItemStorage items) {
-        ScanContext ctx = createContext(cacheAndTypeId(idx).get1(), filePageStore(rootPageId), items);
+        ScanContext ctx = createContext(idx, filePageStore(rootPageId), items);
 
         metaPageVisitor.readAndVisit(rootPageId, ctx);
 
@@ -374,7 +435,7 @@ public class IgniteIndexReader implements AutoCloseable {
      * @return Tree traversal context.
      */
     private ScanContext horizontalTreeScan(long rootPageId, String idx, ItemStorage items) {
-        ScanContext ctx = createContext(cacheAndTypeId(idx).get1(), filePageStore(rootPageId), items);
+        ScanContext ctx = createContext(idx, filePageStore(rootPageId), items);
 
         levelsPageVisitor.readAndVisit(rootPageId, ctx);
 
@@ -485,7 +546,7 @@ public class IgniteIndexReader implements AutoCloseable {
 
         ProgressPrinter progressPrinter = createProgressPrinter("Reading pages sequentially", pagesNum);
 
-        ScanContext ctx = createContext(-1, idxStore, new CountOnlyStorage());
+        ScanContext ctx = createContext(null, idxStore, new CountOnlyStorage());
 
         doWithBuffer((buf, addr) -> {
             for (int i = 0; i < pagesNum; i++) {
@@ -640,25 +701,87 @@ public class IgniteIndexReader implements AutoCloseable {
      * @param name Index name.
      * @return Pair of cache id and type id.
      */
-    public IgnitePair<Integer> cacheAndTypeId(String name) {
+    public GridTuple3<Integer, Integer, String> cacheAndTypeId(String name) {
         return cacheTypeIds.computeIfAbsent(name, k -> {
+            Matcher xId = CACHE_TYPE_ID_INDEX_SEARCH_PATTERN.matcher(k);
+
+            if (xId.find())
+                return new GridTuple3<>(parseInt(xId.group("id")), parseInt(xId.group("typeId")), xId.group("indexName"));
+
             Matcher mId = CACHE_TYPE_ID_SEARCH_PATTERN.matcher(k);
 
             if (mId.find())
-                return new IgnitePair<>(parseInt(mId.group("id")), parseInt(mId.group("typeId")));
+                return new GridTuple3<>(parseInt(mId.group("id")), parseInt(mId.group("typeId")), null);
 
             Matcher cId = CACHE_ID_SEARCH_PATTERN.matcher(k);
 
             if (cId.find())
-                return new IgnitePair<>(parseInt(cId.group("id")), 0);
+                return new GridTuple3<>(parseInt(cId.group("id")), 0, null);
 
-            return new IgnitePair<>(0, 0);
+            return new GridTuple3<>(0, 0, null);
         });
     }
 
     /** */
-    ScanContext createContext(int cacheId, FilePageStore store, ItemStorage items) {
-        return new ScanContext(cacheId, store, items);
+    ScanContext createContext(String idxName, FilePageStore store, ItemStorage items) {
+        GridTuple3<Integer, Integer, String> parsed;
+
+        if (idxName != null)
+            parsed = cacheAndTypeId(idxName);
+        else
+            parsed = new GridTuple3<>(UNKNOWN_CACHE, 0, null);
+
+        return new ScanContext(parsed.get1(), inlineFieldsCount(parsed), store, items);
+    }
+
+    /**
+     * Search index definition inside cache query entities.
+     *
+     * @param parsed Parsed index name.
+     * @return Count of inlined fields or {@code 0} if index definition not found.
+     * @see QueryEntity
+     */
+    protected int inlineFieldsCount(GridTuple3<Integer, Integer, String> parsed) {
+        if (parsed.get1() == UNKNOWN_CACHE || !storedCacheData.containsKey(parsed.get1()))
+            return 0;
+
+        StoredCacheData data = storedCacheData.get(parsed.get1());
+
+        if (Objects.equals(QueryUtils.PRIMARY_KEY_INDEX, parsed.get3())) {
+            if (data.queryEntities().size() > 1) {
+                log.warning("Can't parse inline for PK index when multiple query entities defined for a cache " +
+                    "[idx=" + parsed.get3() + ']');
+
+                return 0;
+            }
+
+            QueryEntity qe = data.queryEntities().iterator().next();
+
+            return qe.getKeyFields() == null ? 1 : qe.getKeyFields().size();
+        }
+
+        QueryIndex idx = null;
+
+        for (QueryEntity qe : data.queryEntities()) {
+            for (QueryIndex idx0 : qe.getIndexes()) {
+                if (Objects.equals(idx0.getName(), parsed.get3())) {
+                    idx = idx0;
+
+                    break;
+                }
+            }
+
+            if (idx != null)
+                break;
+        }
+
+        if (idx == null) {
+            log.warning("Can't find index definition. Inline information not available [idx=" + parsed.get3() + ']');
+
+            return 0;
+        }
+
+        return idx.getFields().size();
     }
 
     /** */
@@ -874,6 +997,27 @@ public class IgniteIndexReader implements AutoCloseable {
 
             log.info(prefix + "---- Count of items found in leaf pages: " + ctx.items.size());
 
+            boolean hasInlineStat = ctx.inline != null && IntStream.of(ctx.inline).anyMatch(i -> i > 0);
+
+            if (hasInlineStat) {
+                log.info(prefix + "---- Inline usage statistics [inlineSize=" + ctx.inline.length + " bytes]");
+
+                List<List<?>> data = new ArrayList<>(ctx.inline.length);
+                for (int i = 0; i < ctx.inline.length; i++) {
+                    if (ctx.inline[i] == 0)
+                        continue;
+
+                    data.add(Arrays.asList(prefix, i + 1, ctx.inline[i]));
+                }
+
+                SystemViewCommand.printTable(
+                    Arrays.asList(prefix, "Used bytes", "Entries count"),
+                    Arrays.asList(STRING, NUMBER, NUMBER),
+                    data,
+                    log
+                );
+            }
+
             printErrors(
                 prefix,
                 "Errors:",
@@ -884,8 +1028,10 @@ public class IgniteIndexReader implements AutoCloseable {
 
             totalErr += ctx.errors.size();
 
+            GridTuple3<Integer, Integer, String> parsed = cacheAndTypeId(idxName);
+
             cacheIdxSizes
-                .computeIfAbsent(cacheAndTypeId(idxName), k -> new HashMap<>())
+                .computeIfAbsent(new IgnitePair<>(parsed.get1(), parsed.get2()), k -> new HashMap<>())
                 .put(idxName, ctx.items.size());
         }
 
@@ -1040,6 +1186,33 @@ public class IgniteIndexReader implements AutoCloseable {
         );
     }
 
+    /**
+     * Creating new {@link FilePageStore} and initializing it.
+     * It can return {@code null} if partition file were not found, for example: node should not contain it by affinity.
+     *
+     * @param partId Partition ID.
+     * @param type Data type, can be {@link PageIdAllocator#FLAG_IDX} or {@link PageIdAllocator#FLAG_DATA}.
+     * @param storeFactory Store factory.
+     * @return New instance of {@link FilePageStore} or {@code null}.
+     * @throws IgniteCheckedException If there are errors when creating or initializing {@link FilePageStore}.
+     */
+    @Nullable private FilePageStore filePageStore(
+        int partId,
+        byte type,
+        FileVersionCheckingFactory storeFactory
+    ) throws IgniteCheckedException {
+        File file = new File(root, partId == INDEX_PARTITION ? INDEX_FILE_NAME : format(PART_FILE_TEMPLATE, partId));
+
+        if (!file.exists())
+            return null;
+
+        FilePageStore filePageStore = (FilePageStore)storeFactory.createPageStore(type, file, l -> {});
+
+        filePageStore.ensure();
+
+        return filePageStore;
+    }
+
     /** {@inheritDoc} */
     @Override public void close() throws StorageException {
         idxStore.stop(false);
@@ -1175,6 +1348,9 @@ public class IgniteIndexReader implements AutoCloseable {
 
             BPlusLeafIO<?> io = PageIO.getPageIO(addr);
 
+            if (io instanceof AbstractInlineLeafIO)
+                visitInline(addr, (AbstractInlineLeafIO)io, ctx);
+
             doWithoutErrors(() -> {
                 for (int i = 0; i < io.getCount(addr); i++) {
                     if (io instanceof IndexStorageImpl.MetaStoreLeafIO)
@@ -1187,6 +1363,65 @@ public class IgniteIndexReader implements AutoCloseable {
             ctx.onLeafPage(PageIO.getPageId(addr), items);
         }
 
+        /** */
+        private void visitInline(long addr, AbstractInlineLeafIO io, ScanContext ctx) {
+            int inlineSz = ((InlineIO)io).inlineSize();
+
+            if (ctx.inlineFldCnt == 0)
+                return;
+
+            if (ctx.inline == null)
+                ctx.inline = new int[inlineSz];
+
+            IndexKeyTypeSettings settings = new IndexKeyTypeSettings();
+
+            for (int i = 0; i < io.getCount(addr); i++) {
+                int itemOff = io.offset(i);
+                int realInlineSz = 0;
+                int fldCnt = 0;
+
+                while (realInlineSz < inlineSz && fldCnt < ctx.inlineFldCnt) {
+                    int type0 = PageUtils.getByte(addr, itemOff + realInlineSz);
+
+                    if (type0 == IndexKeyTypes.UNKNOWN || type0 == IndexKeyTypes.NULL) {
+                        realInlineSz += 1;
+
+                        continue;
+                    }
+
+                    InlineIndexKeyType type = InlineIndexKeyTypeRegistry.get(type0, settings);
+
+                    if (type == null) {
+                        log.log(Level.FINEST, "Unknown inline type [type=" + type0 + ']');
+
+                        break;
+                    }
+
+                    if (type.keySize() == UNKNOWN_CACHE) {
+                        try {
+                            // Assuming all variable length keys written using `writeBytes` method.
+                            byte[] bytes = NullableInlineIndexKeyType.readBytes(addr, itemOff + realInlineSz);
+
+                            realInlineSz += Short.BYTES; /* size of the array is short number. */
+                            realInlineSz += bytes.length;
+                        }
+                        catch (Throwable e) {
+                            log.warning("Error while reading inline [msg=" + e.getMessage() + ']');
+
+                            break;
+                        }
+                    }
+                    else
+                        realInlineSz += type.keySize();
+
+                    realInlineSz++; // One more byte for type.
+                    fldCnt++;
+                }
+
+                ctx.inline[realInlineSz - 1]++;
+            }
+        }
+
         /** */
         private Object leafItem(BPlusLeafIO<?> io, long addr, int idx, ScanContext ctx) {
             if (!(io instanceof InlineIO || io instanceof PendingRowIO || io instanceof RowLinkIO))
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderFilePageStoreFactory.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderFilePageStoreFactory.java
deleted file mode 100644
index c4b1518b8ac..00000000000
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderFilePageStoreFactory.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.commandline.indexreader;
-
-import java.io.File;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.pagemem.PageIdAllocator;
-import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
-import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
-import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
-import org.apache.ignite.internal.processors.cache.persistence.file.FileVersionCheckingFactory;
-import org.jetbrains.annotations.Nullable;
-
-import static java.lang.String.format;
-import static java.util.Objects.nonNull;
-import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
-import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX;
-import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME;
-import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE;
-
-/**
- * Factory {@link FilePageStore} for analyzing partition and index files.
- */
-class IgniteIndexReaderFilePageStoreFactory {
-    /** Directory with data(partitions and index). */
-    private final File dir;
-
-    /** {@link FilePageStore} factory by page store version. */
-    private final FileVersionCheckingFactory storeFactory;
-
-    /** Page size. */
-    private final int pageSize;
-
-    /** Partition count. */
-    private final int partCnt;
-
-    /**
-     * Constructor.
-     *
-     * @param dir Directory with data(partitions and index).
-     * @param pageSize Page size.
-     * @param partCnt Partition count.
-     * @param filePageStoreVer Page store version.
-     */
-    IgniteIndexReaderFilePageStoreFactory(File dir, int pageSize, int partCnt, int filePageStoreVer) {
-        this.dir = dir;
-        this.pageSize = pageSize;
-        this.partCnt = partCnt;
-
-        storeFactory = new FileVersionCheckingFactory(
-            new AsyncFileIOFactory(),
-            new AsyncFileIOFactory(),
-            () -> pageSize
-        ) {
-            /** {@inheritDoc} */
-            @Override public int latestVersion() {
-                return filePageStoreVer;
-            }
-        };
-    }
-
-    /**
-     * Creating new {@link FilePageStore} and initializing it.
-     * It can return {@code null} if partition file were not found, for example: node should not contain it by affinity.
-     *
-     * @param partId Partition ID.
-     * @param type Data type, can be {@link PageIdAllocator#FLAG_IDX} or {@link PageIdAllocator#FLAG_DATA}.
-     * @return New instance of {@link FilePageStore} or {@code null}.
-     * @throws IgniteCheckedException If there are errors when creating or initializing {@link FilePageStore}.
-     */
-    @Nullable FilePageStore createFilePageStore(int partId, byte type) throws IgniteCheckedException {
-        File file = getFile(dir, partId, null);
-
-        if (!file.exists())
-            return null;
-
-        FilePageStore filePageStore = (FilePageStore)storeFactory.createPageStore(type, file, l -> {});
-
-        filePageStore.ensure();
-
-        return filePageStore;
-    }
-
-    /**
-     * Getting a partition or index file that may not exist.
-     *
-     * @param dir Directory to get partition or index file.
-     * @param partId ID of partition or index.
-     * @param fileExt File extension if it differs from {@link FilePageStoreManager#FILE_SUFFIX}.
-     * @return Partition or index file that may not exist.
-     */
-    private File getFile(File dir, int partId, @Nullable String fileExt) {
-        String fileName = partId == INDEX_PARTITION ? INDEX_FILE_NAME : format(PART_FILE_TEMPLATE, partId);
-
-        if (nonNull(fileExt) && !FILE_SUFFIX.equals(fileExt))
-            fileName = fileName.replace(FILE_SUFFIX, fileExt);
-
-        return new File(dir, fileName);
-    }
-
-    /**
-     * Return page size.
-     *
-     * @return Page size.
-     */
-    int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * Return partition count.
-     *
-     * @return Partition count.
-     */
-    int partitionCount() {
-        return partCnt;
-    }
-}
diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/ScanContext.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/ScanContext.java
index bb1ef520533..16074d31b44 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/ScanContext.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/indexreader/ScanContext.java
@@ -31,6 +31,9 @@ class ScanContext {
     /** Cache id or {@code -1} for sequential scan. */
     final int cacheId;
 
+    /** Count of inline fields. */
+    final int inlineFldCnt;
+
     /** Page store. */
     final FilePageStore store;
 
@@ -43,9 +46,17 @@ class ScanContext {
     /** List of items storage. */
     final ItemStorage items;
 
+    /**
+     * Inline usage statistics.
+     * Size of the array equal index inline size.
+     * Each cell contains count of item that use exact number of inline bytes.
+     */
+    int[] inline;
+
     /** */
-    public ScanContext(int cacheId, FilePageStore store, ItemStorage items) {
+    public ScanContext(int cacheId, int inlineFldCnt, FilePageStore store, ItemStorage items) {
         this.cacheId = cacheId;
+        this.inlineFldCnt = inlineFldCnt;
         this.store = store;
         this.items = items;
         this.stats = new LinkedHashMap<>();
diff --git a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
index f6d78008a58..db12502f4e2 100644
--- a/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
+++ b/modules/control-utility/src/test/java/org/apache/ignite/internal/commandline/indexreader/IgniteIndexReaderTest.java
@@ -58,6 +58,7 @@ import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.util.GridStringBuilder;
 import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -131,7 +132,10 @@ public class IgniteIndexReaderTest extends GridCommandHandlerAbstractTest {
             LINE_DELIM + "<PREFIX>---- Page stat:" +
             LINE_DELIM + "<PREFIX>Type.*Pages.*Free space.*" +
             LINE_DELIM + "<PREFIX>([0-9a-zA-Z]{1,50}.*[0-9]{1,5}" +
-            LINE_DELIM + "<PREFIX>){%s,1000}---- Count of items found in leaf pages: %s" +
+            LINE_DELIM + "<PREFIX>){%s,1000}---- Count of items found in leaf pages: %s(" +
+            LINE_DELIM + "<PREFIX>---- Inline usage statistics \\[inlineSize=[0-9]{1,3} bytes\\]" +
+            LINE_DELIM + "<PREFIX>.*Used bytes.*Entries count(" +
+            LINE_DELIM + "<PREFIX>.*[0-9]{1,10}){0,64})?" +
             LINE_DELIM;
 
     /** Regexp to validate output of correct index. */
@@ -281,9 +285,16 @@ public class IgniteIndexReaderTest extends GridCommandHandlerAbstractTest {
         // Take any inner page from tree.
         AtomicLong anyLeafId = new AtomicLong();
 
-        IgniteIndexReader reader0 = new IgniteIndexReader(null, false, createFilePageStoreFactory(dir), createTestLogger()) {
-            @Override ScanContext createContext(int cacheId, FilePageStore store, ItemStorage items) {
-                return new ScanContext(cacheId, store, items) {
+        IgniteIndexReader reader0 = new IgniteIndexReader(PAGE_SIZE, PART_CNT, PAGE_STORE_VER, dir, null, false, createTestLogger()) {
+            @Override ScanContext createContext(String idxName, FilePageStore store, ItemStorage items) {
+                GridTuple3<Integer, Integer, String> parsed;
+
+                if (idxName != null)
+                    parsed = cacheAndTypeId(idxName);
+                else
+                    parsed = new GridTuple3<>(UNKNOWN_CACHE, 0, null);
+
+                return new ScanContext(parsed.get1(), inlineFieldsCount(parsed), store, items) {
                     @Override public void onLeafPage(long pageId, List<Object> data) {
                         super.onLeafPage(pageId, data);
 
@@ -719,9 +730,12 @@ public class IgniteIndexReaderTest extends GridCommandHandlerAbstractTest {
         Logger logger = createTestLogger();
 
         IgniteIndexReader reader0 = new IgniteIndexReader(
+            PAGE_SIZE,
+            PART_CNT,
+            PAGE_STORE_VER,
+            new File(workDir, dataDir(cacheGrp)),
             isNull(idxs) ? null : idx -> Arrays.stream(idxs).anyMatch(idx::endsWith),
             checkParts,
-            createFilePageStoreFactory(new File(workDir, dataDir(cacheGrp))),
             logger
         ) {
             /** {@inheritDoc} */
@@ -739,17 +753,6 @@ public class IgniteIndexReaderTest extends GridCommandHandlerAbstractTest {
         return testOut.toString();
     }
 
-    /**
-     * Create new {@link IgniteIndexReaderFilePageStoreFactory}.
-     *
-     * @param dir Data rirectory.
-     */
-    protected IgniteIndexReaderFilePageStoreFactory createFilePageStoreFactory(
-        File dir
-    ) {
-        return new IgniteIndexReaderFilePageStoreFactory(dir, PAGE_SIZE, PART_CNT, PAGE_STORE_VER);
-    }
-
     /**
      * Test checks correctness of index.
      *
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java
index a39b71f8e39..c95cafa0931 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cache/query/index/sorted/inline/types/NullableInlineIndexKeyType.java
@@ -172,7 +172,7 @@ public abstract class NullableInlineIndexKeyType<T extends IndexKey> implements
     protected abstract @Nullable T get0(long pageAddr, int off);
 
     /** Read variable length bytearray */
-    protected byte[] readBytes(long pageAddr, int off) {
+    public static byte[] readBytes(long pageAddr, int off) {
         int size = PageUtils.getShort(pageAddr, off + 1) & 0x7FFF;
         return PageUtils.getBytes(pageAddr, off + 3, size);
     }