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 2020/04/20 07:45:30 UTC

[GitHub] [ignite] NSAmelchev opened a new pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

NSAmelchev opened a new pull request #7693:
URL: https://github.com/apache/ignite/pull/7693


   Cluster profiling tool based on performance logging to separate log category.
   How to use:
   1. Configurate profiling log output to separate file.
   2. Run cluster and workload. Stop the cluster.
   3. Build the report using profiling.sh. 
   
   The report contains statistics:
   - cache operations: `get|getAll|put|putAll|remove|removeAll|getAndPut|getAndRemove|lock|invoke|invokeAll`
   - transaction commit, rollback; histogram of transaction durations
   - SQL, SCAN queries, top of slowest queries
   - Tasks and jobs. Top of slowest tasks.
   


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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449030785



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)

Review comment:
       I want to avoid collisions. It may lead to wrong results interpretation.
   `MAX_CACHED_STRING_COUNT` can be decreased to reduce GC pressure.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451482027



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            // Stop write new data.
+            if (buf != null)
+                buf.close();
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            if (buf != null)
+                buf.free();
+
+            U.closeQuiet(fileIo);
+
+            readyForFlushSize.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer ringBuf = ringByteBuffer;
+
+        // Starting.
+        if (ringBuf == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true)) {
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+
+        int readySize = readyForFlushSize.addAndGet(size);

Review comment:
       We have a race here.
   
   1. FileWriter - writes some data and decrease `readyForFlushSize`.
   2. Other threads executes `doWrite` and constantly increase `readySize` so it's become bigger than `DFLT_FLUSH_SIZE`. This leads that those threads will have contention of `wakeUp` synchronization.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449488185



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449227192



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(text.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(taskName.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443532877



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/profiling/IgniteProfilingMBean.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.profiling;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.mxbean.MXBeanDescription;
+import org.apache.ignite.mxbean.MXBeanParameter;
+
+/**
+ * MBean provide access to profiling management.
+ */
+@MXBeanDescription("MBean provide access to profiling management.")
+public interface IgniteProfilingMBean {
+    /**
+     * Start profiling in the cluster with default settings.
+     *
+     * @see FileProfiling#DFLT_FILE_MAX_SIZE
+     * @see FileProfiling#DFLT_BUFFER_SIZE
+     * @see FileProfiling#DFLT_FLUSH_SIZE
+     */
+    @MXBeanDescription("Start profiling in the cluster.")
+    public void startProfiling() throws IgniteCheckedException;

Review comment:
       The word 'profiling' exists in each method of this bean. Can we remove it? `start`, `stop`, `enabled` seems less verbose.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446078586



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void walkFile(Path file, IgnitePerformanceStatistics... handlers) throws IOException {
+        ByteBuffer buf = allocateDirect(READ_BUFFER_SIZE).order(nativeOrder());
+
+        try (
+            FileIO io = ioFactory.create(file.toFile());
+            PerformanceStatisticsDeserializer des = new PerformanceStatisticsDeserializer(handlers)
+        ) {
+            while (true) {
+                int read = io.read(buf);
+
+                buf.flip();
+
+                if (read <= 0)
+                    break;
+
+                while (true) {

Review comment:
       Let's rewrite it to the `while (des.deserialize(buf));`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448864329



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration, boolean commit);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime, long duration,
+        boolean success);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads,
+        long physicalReads);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut);
+
+    /** Cache operations types. */
+    public enum CacheOperationType {

Review comment:
       Let's make this enum not internal. 




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r487789242



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.internal.processors.performancestatistics.AbstractPerformanceStatisticsTest.LoadNode.CLIENT;
+import static org.apache.ignite.internal.processors.performancestatistics.AbstractPerformanceStatisticsTest.LoadNode.SERVER;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@RunWith(Parameterized.class)
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Load node to run operations from. */
+    @Parameterized.Parameter
+    public LoadNode loadNode;
+
+    /** @return Test parameters. */
+    @Parameterized.Parameters(name = "loadNode={0}")
+    public static Collection<?> parameters() {
+        return Arrays.asList(new Object[][] {{SERVER}, {CLIENT}});
+    }
+
+    /** Ignite. */
+    private static IgniteEx srv;
+
+    /** Ignite node to run load from. */
+    private static IgniteEx node;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        srv = startGrid(0);
+
+        IgniteEx client = startClientGrid(1);
+
+        node = loadNode == SERVER ? srv : client;
+
+        cache = node.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            node.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, cnt) -> cnt == null ? 1 : ++cnt);
+
+                tasks.incrementAndGet();
+
+                assertEquals(node.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446046198



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {
+        return writer;
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean statisticsEnabled() {
+        return writer.performanceStatisticsEnabled();
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @return Future to be completed on collecting started.
+     */
+    public IgniteInternalFuture<Void> startStatistics() {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {
+        return writer;
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean statisticsEnabled() {
+        return writer.performanceStatisticsEnabled();
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @return Future to be completed on collecting started.
+     */
+    public IgniteInternalFuture<Void> startStatistics() {
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS)) {
+            return new GridFinishedFuture<>(
+                new IllegalStateException("Not all nodes in the cluster support collecting performance statistics."));
+        }
+
+        GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
+
+        UUID uuid = UUID.randomUUID();
+
+        synchronized (mux) {
+            if (disconnected || stopped) {
+                return new GridFinishedFuture<>(
+                    new IgniteFutureCancelledException("Node " + (stopped ? "stopped" : "disconnected")));
+            }
+
+            reqFuts.put(uuid, fut);
+        }
+
+        proc.start(uuid, true);
+
+        return fut;
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @return Future to be completed on collecting stopped.
+     */
+    public IgniteInternalFuture<Void> stopStatistics() {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r444386440



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -6794,6 +6899,55 @@ public InvokeAllTimeStatClosure(CacheMetricsImpl metrics, final long start) {
         }
     }
 
+    /** */
+    private class ProfileClosure<T> implements CI1<IgniteInternalFuture<T>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Operation type. */
+        private final CacheOperationType op;
+
+        /** Start time in nanoseconds. */
+        private final long start;
+
+        /**
+         * @param op Operation type.
+         * @param start Start time in nanoseconds.
+         */
+        public ProfileClosure(CacheOperationType op, long start) {
+            this.op = op;
+            this.start = start;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteInternalFuture<T> fut) {
+            try {
+                if (!fut.isCancelled()) {
+                    fut.get();

Review comment:
       Removed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443686343



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
##########
@@ -3190,6 +3198,23 @@ private void collectInfo() {
         }
     }
 
+    /**
+     * Profiles transaction.
+     *
+     * @param tx Transaction.
+     * @param commit {@code True} if transaction commited.

Review comment:
       Typo: flag name should be `committed` or `isCommitted`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446006908



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -2248,6 +2248,15 @@ private void onCacheStarted(GridCacheContext cacheCtx) throws IgniteCheckedExcep
                 ", mvcc=" + cacheCtx.mvccEnabled() + ']');
         }
 
+        if (ctx.metric().profilingEnabled()) {

Review comment:
       Excluded.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449433144



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return {@code True} if string hash code is cached. {@code False} if need write string.  */
+        boolean stringCached(String str) {
+            boolean cached = cachedStrings.contains(str.hashCode());
+
+            if (!cached)
+                cachedStrings.add(str.hashCode());
+
+            return cached;
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {

Review comment:
       IMHO, this is `FileWriter` entity. It uses private fields.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return {@code True} if string hash code is cached. {@code False} if need write string.  */
+        boolean stringCached(String str) {
+            boolean cached = cachedStrings.contains(str.hashCode());
+
+            if (!cached)
+                cachedStrings.add(str.hashCode());
+
+            return cached;
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        notify();

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446046342



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {

Review comment:
       Writer encapsulated.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r447032382



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter implements IgnitePerformanceStatistics {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatisticsWriter writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatisticsWriter(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {

Review comment:
       Why do we need `DistributedProcess` here?
   What is the "process" here in the first place?
   Let's implement this as 'DistributedMetaStorage' listener.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r444386791



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
##########
@@ -3190,6 +3198,23 @@ private void collectInfo() {
         }
     }
 
+    /**
+     * Profiles transaction.
+     *
+     * @param tx Transaction.
+     * @param commit {@code True} if transaction commited.

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449564308



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                // Required to start writing data to the file.
+                notify();
+            }
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+        }
+
+        /** Logs warning message and stops collecting statistics. */
+        void onMaxFileSizeReached() {

Review comment:
       This method used only once. Let's inline 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r487701521



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,305 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.internal.processors.performancestatistics.AbstractPerformanceStatisticsTest.LoadNode.CLIENT;
+import static org.apache.ignite.internal.processors.performancestatistics.AbstractPerformanceStatisticsTest.LoadNode.SERVER;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@RunWith(Parameterized.class)
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Load node to run operations from. */
+    @Parameterized.Parameter
+    public LoadNode loadNode;
+
+    /** @return Test parameters. */
+    @Parameterized.Parameters(name = "loadNode={0}")
+    public static Collection<?> parameters() {
+        return Arrays.asList(new Object[][] {{SERVER}, {CLIENT}});
+    }
+
+    /** Ignite. */
+    private static IgniteEx srv;
+
+    /** Ignite node to run load from. */
+    private static IgniteEx node;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        srv = startGrid(0);
+
+        IgniteEx client = startClientGrid(1);
+
+        node = loadNode == SERVER ? srv : client;
+
+        cache = node.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            node.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, cnt) -> cnt == null ? 1 : ++cnt);
+
+                tasks.incrementAndGet();
+
+                assertEquals(node.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);

Review comment:
       Let's store `System.currentTimeMillis` before running jobs and here check.
   `assertTrue(startTime >= jobsStartTime);`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449192874



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);

Review comment:
       Let's introduce the following method to generalize write:
   
   ```
   public void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
           FileWriter writer = fileWriter;
   
           if (writer == null)
               return;
   
           int size = sizeSupplier.apply();
           SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(op, size);
   
           if (seg == null)
               return;
   
           writer.apply(seg.buffer());
           seg.release();
   }
   ```




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r439996151



##########
File path: modules/profiling/src/main/java/org/apache/ignite/internal/profiling/util/ProfilingDeserializer.java
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.profiling.util;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.profiling.IgniteProfiling;
+import org.apache.ignite.internal.profiling.IgniteProfiling.CacheOperationType;
+import org.apache.ignite.internal.profiling.LogFileProfiling;
+import org.apache.ignite.internal.profiling.LogFileProfiling.OperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static org.apache.ignite.internal.profiling.LogFileProfiling.readIgniteUuid;
+import static org.apache.ignite.internal.profiling.LogFileProfiling.readUuid;
+
+/**
+ * Profiling operations deserializer.
+ *
+ * @see LogFileProfiling
+ */
+public class ProfilingDeserializer implements AutoCloseable {
+    /** Cached strings by id. */
+    private final ConcurrentHashMap<Short, String> stringById = new ConcurrentHashMap<>();
+
+    /** Handlers to process deserialized operation. */
+    private final IgniteProfiling[] handlers;
+
+    /** @param handlers Handlers to process deserialized operation. */
+    public ProfilingDeserializer(IgniteProfiling... handlers) {
+        this.handlers = handlers;
+    }
+
+    /**
+     * Tries to deserialize profiling operation from buffer.
+     *
+     * @param buf Buffer.
+     * @return {@code True} if operation parsed. {@code False} if not enough bytes.
+     */
+    public boolean deserialize(ByteBuffer buf) {
+        int pos = buf.position();
+
+        if (buf.remaining() < 1)
+            return false;
+
+        byte opTypeByte = buf.get();
+
+        OperationType opType = OperationType.fromOrdinal(opTypeByte);
+
+        switch (opType) {
+            case CACHE_OPERATION: {
+                if (buf.remaining() < 1 + 4 + 8 + 8)

Review comment:
       I put the writer and iterator in one place. See `FileProfiling` and `FileProfilingWalker`. Add tests. See `AbstractProfilingTest` implementations.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r455654685



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsQueryTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.Query;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SCAN;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
+
+/** Tests query performance statistics. */
+public class PerformanceStatisticsQueryTest extends AbstractPerformanceStatisticsTest {
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;

Review comment:
       We should add tests with `pageSize` less than the entire result set to ensure that with the several pages we track all query results correctly.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451474344



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);

Review comment:
       Let's append to the file if it exists.
   It seems all we need to do is remove this line.
   
   We need a ticket to provide `clear` command, also.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446109382



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -198,54 +201,74 @@ protected void removeQueryFuture(long reqId) {
         assert req.mvccSnapshot() != null || !cctx.mvccEnabled() || req.cancel() ||
             (req.type() == null && !req.fields()) : req; // Last assertion means next page request.
 
-        if (req.cancel()) {
-            cancelIds.add(new CancelMessageId(req.id(), sndId));
+        boolean performanceStatsEnabled = cctx.kernalContext().performanceStatistics().enabled();
 
-            if (req.fields())
-                removeFieldsQueryResult(sndId, req.id());
-            else
-                removeQueryResult(sndId, req.id());
-        }
-        else {
-            if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
-                if (!F.eq(req.cacheName(), cctx.name())) {
-                    GridCacheQueryResponse res = new GridCacheQueryResponse(
-                        cctx.cacheId(),
-                        req.id(),
-                        new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
-                            ", actual=" + req.cacheName()),
-                        cctx.deploymentEnabled());
+        if (performanceStatsEnabled)
+            IoStatisticsQueryHelper.startGatheringQueryStatistics();
 
-                    sendQueryResponse(sndId, res, 0);
-                }
-                else {
-                    threads.put(req.id(), Thread.currentThread());
+        try {
+            if (req.cancel()) {
+                cancelIds.add(new CancelMessageId(req.id(), sndId));
 
-                    try {
-                        GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
+                if (req.fields())
+                    removeFieldsQueryResult(sndId, req.id());
+                else
+                    removeQueryResult(sndId, req.id());
+            }
+            else {

Review comment:
       Typo: this can be rewritten as `else if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {` to reduce one indentation level.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454928429



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Ignite. */
+    private static IgniteEx ignite;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        log = new ListeningTestLogger(GridAbstractTest.log);
+
+        ignite = startGrid(0);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            ignite.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);
+
+                tasks.incrementAndGet();
+
+                assertEquals(ignite.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);
+                assertTrue(duration >= 0);
+                assertEquals(-1, affPartId);
+            }
+
+            @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);

Review comment:
       `val` should be renamed to `cnt` here and below.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446138038



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMbeanImpl.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * {@link IgnitePerformanceStatisticsMBean} implementation.
+ */
+public class IgnitePerformanceStatisticsMbeanImpl implements IgnitePerformanceStatisticsMBean {

Review comment:
       Let's rename this to `PerformanceStatisticsMBeanImpl`.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448989511



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {

Review comment:
       No need. Removed.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449138640



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449403650



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;

Review comment:
       Let's use `IgniteUtils#MB` here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449210311



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {

Review comment:
       It seems, `stop` results are ignored.
   Can we make this method to return `void`?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449206659



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(text.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(taskName.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;

Review comment:
       This constant not used now. Please, remove 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443601200



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -3324,6 +3418,9 @@ protected void removeAll0(final Collection<? extends K> keys) throws IgniteCheck
         if (statsEnabled)
             fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
 
+        if (profilingEnabled)
+            fut.listen(new ProfileClosure<>(CacheOperationType.REMOVE_ALL, start));

Review comment:
       We should have separate class for each operations so we can eliminate `operation` variable from `ProfileClosure`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r487700210



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.lang.management.ThreadInfo;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.mxbean.PerformanceStatisticsMBean;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.PERF_STAT_DIR;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.WRITER_THREAD_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Ignite performance statistics abstract test.
+ */
+public abstract class AbstractPerformanceStatisticsTest extends GridCommonAbstractTest {
+    /** */
+    public static final long TIMEOUT = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+
+        U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, true);
+    }
+
+    /** Starts collecting performance statistics. */
+    protected static void startCollectStatistics() throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        statisticsMBean(grids.get(0).name()).start();
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** Stops and reads collecting performance statistics. */
+    protected static void stopCollectStatisticsAndRead(TestHandler... handlers) throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        statisticsMBean(grids.get(0).name()).stop();
+
+        waitForStatisticsEnabled(false);
+
+        File dir = U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false);
+
+        new FilePerformanceStatisticsReader(handlers).read(Collections.singletonList(dir));
+    }
+
+    /** Wait for statistics started/stopped in the cluster. */
+    protected static void waitForStatisticsEnabled(boolean performanceStatsEnabled) throws Exception {
+        assertTrue(waitForCondition(() -> {
+            List<Ignite> grids = G.allGrids();
+
+            for (Ignite grid : grids)
+                if (performanceStatsEnabled != statisticsMBean(grid.name()).started())
+                    return false;
+
+            // Make sure that writer flushed data and stopped.
+            if (!performanceStatsEnabled) {
+                for (long id : U.getThreadMx().getAllThreadIds()) {
+                    ThreadInfo info = U.getThreadMx().getThreadInfo(id);
+
+                    if (info != null && info.getThreadState() != Thread.State.TERMINATED &&
+                        info.getThreadName().startsWith(WRITER_THREAD_NAME))
+                        return false;
+                }
+            }
+
+            return true;
+        }, TIMEOUT));
+    }
+
+    /**
+     * @param igniteInstanceName Ignite instance name.
+     * @return Ignite performance statistics MBean.
+     */
+    protected static PerformanceStatisticsMBean statisticsMBean(String igniteInstanceName) {
+        return getMxBean(igniteInstanceName, "PerformanceStatistics", PerformanceStatisticsMBeanImpl.class,
+            PerformanceStatisticsMBean.class);
+    }
+
+    /** Test performance statistics handler. */
+    public class TestHandler implements PerformanceStatisticsHandler {
+        /** {@inheritDoc} */
+        @Override public void cacheOperation(UUID nodeId, OperationType type, int cacheId, long startTime, long duration) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration,
+            boolean commited) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime,
+            long duration, boolean success) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id,
+            long logicalReads, long physicalReads) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+            int affPartId) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {
+            // No-op.
+        }
+    }
+
+    /** Node to run load from. */
+    enum LoadNode {

Review comment:
       LoadNode -> ClientType




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448876413



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration, boolean commit);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime, long duration,
+        boolean success);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads,
+        long physicalReads);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut);
+
+    /** Cache operations types. */
+    public enum CacheOperationType {

Review comment:
       > Let's make this enum not internal.
   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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446101796



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -198,54 +201,74 @@ protected void removeQueryFuture(long reqId) {
         assert req.mvccSnapshot() != null || !cctx.mvccEnabled() || req.cancel() ||
             (req.type() == null && !req.fields()) : req; // Last assertion means next page request.
 
-        if (req.cancel()) {
-            cancelIds.add(new CancelMessageId(req.id(), sndId));
+        boolean performanceStatsEnabled = cctx.kernalContext().performanceStatistics().enabled();
 
-            if (req.fields())
-                removeFieldsQueryResult(sndId, req.id());
-            else
-                removeQueryResult(sndId, req.id());
-        }
-        else {
-            if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
-                if (!F.eq(req.cacheName(), cctx.name())) {
-                    GridCacheQueryResponse res = new GridCacheQueryResponse(
-                        cctx.cacheId(),
-                        req.id(),
-                        new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
-                            ", actual=" + req.cacheName()),
-                        cctx.deploymentEnabled());
+        if (performanceStatsEnabled)
+            IoStatisticsQueryHelper.startGatheringQueryStatistics();
 
-                    sendQueryResponse(sndId, res, 0);
-                }
-                else {
-                    threads.put(req.id(), Thread.currentThread());
+        try {
+            if (req.cancel()) {
+                cancelIds.add(new CancelMessageId(req.id(), sndId));
 
-                    try {
-                        GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
+                if (req.fields())
+                    removeFieldsQueryResult(sndId, req.id());
+                else
+                    removeQueryResult(sndId, req.id());
+            }
+            else {
+                if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
+                    if (!F.eq(req.cacheName(), cctx.name())) {
+                        GridCacheQueryResponse res = new GridCacheQueryResponse(
+                            cctx.cacheId(),
+                            req.id(),
+                            new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
+                                ", actual=" + req.cacheName()),
+                            cctx.deploymentEnabled());
+
+                        sendQueryResponse(sndId, res, 0);
+                    }
+                    else {
+                        threads.put(req.id(), Thread.currentThread());
 
-                        if (info == null)
-                            return;
+                        try {
+                            GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
 
-                        if (req.fields())
-                            runFieldsQuery(info);
-                        else
-                            runQuery(info);
-                    }
-                    catch (Throwable e) {
-                        U.error(log(), "Failed to run query.", e);
+                            if (info == null)
+                                return;
+
+                            if (req.fields())
+                                runFieldsQuery(info);
+                            else
+                                runQuery(info);
+                        }
+                        catch (Throwable e) {
+                            U.error(log(), "Failed to run query.", e);
 
-                        sendQueryResponse(sndId, new GridCacheQueryResponse(cctx.cacheId(), req.id(), e.getCause(),
-                            cctx.deploymentEnabled()), 0);
+                            sendQueryResponse(sndId, new GridCacheQueryResponse(cctx.cacheId(), req.id(), e.getCause(),
+                                cctx.deploymentEnabled()), 0);
 
-                        if (e instanceof Error)
-                            throw (Error)e;
-                    }
-                    finally {
-                        threads.remove(req.id());
+                            if (e instanceof Error)
+                                throw (Error)e;
+                        }
+                        finally {
+                            threads.remove(req.id());
+                        }
                     }
                 }
             }
+        } finally {

Review comment:
       Typo: finally should be on new line.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449394399



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {

Review comment:
       this method used only once. Let's inline 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443687740



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
##########
@@ -234,6 +241,9 @@
     /** Nonheap memory metrics. */
     private final MemoryUsageMetrics nonHeap;
 
+    /** Profiling. */
+    private final FileProfiling profiling;

Review comment:
       It seems, code that related to performance statistics should be moved to separate Manager.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449562489



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {

Review comment:
       Let's rename this to `stop`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448857734



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,193 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsHandler.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatisticsWriter writer;
+
+    /** Metastorage with the write access. */
+    private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatisticsWriter(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    boolean start = (boolean)newVal;
+
+                    if (start)
+                        ctx.closure().runLocalSafe(writer::start);
+                    else
+                        writer.stop();
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean enabled = metastorage.read(STAT_ENABLED_PREFIX);
+
+                    if (enabled != null && enabled)
+                        ctx.closure().runLocalSafe(writer::start);
+                    else
+                        writer.stop();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return writer.performanceStatisticsEnabled();
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {

Review comment:
       Let's rename this to `start`.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454946050



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Ignite. */
+    private static IgniteEx ignite;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        log = new ListeningTestLogger(GridAbstractTest.log);
+
+        ignite = startGrid(0);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            ignite.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);
+
+                tasks.incrementAndGet();
+
+                assertEquals(ignite.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);
+                assertTrue(duration >= 0);
+                assertEquals(-1, affPartId);
+            }
+
+            @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453770838



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454312138



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.jobRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.queryReadsRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.queryRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.taskRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionRecordSize;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = (int)(8 * U.MB);
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN =
+        "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /** File name pattern. */
+    private static final Pattern FILE_PATTERN = Pattern.compile("^node-(" + UUID_STR_PATTERN + ").prf$");
+
+    /** IO factory. */
+    private final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /** Handlers to process deserialized operations. */
+    private final PerformanceStatisticsHandler[] handlers;
+
+    /** @param handlers Handlers to process deserialized operations. */
+    FilePerformanceStatisticsReader(PerformanceStatisticsHandler... handlers) {
+        A.ensure(handlers != null, "At least one handler expected.");

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449399049



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {

Review comment:
       This method should be moved to the reader of some utility class.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449031663



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)

Review comment:
       What collisions do you have in mind? A collision of hash codes of the two strings?




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454189166



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = (int)(8 * U.MB);
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN =
+        "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /** File name pattern. */
+    private static final Pattern FILE_PATTERN = Pattern.compile("^node-(" + UUID_STR_PATTERN + ").prf$");
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics files.
+     *
+     * @param filesOrDirs Files or directories.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void read(List<File> filesOrDirs, PerformanceStatisticsHandler... handlers) throws IOException {
+        List<File> files = resolveFiles(filesOrDirs);
+
+        if (files.isEmpty())
+            return;
+
+        for (File file : files)
+            readFile(file, handlers);
+    }
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    private static void readFile(File file, PerformanceStatisticsHandler... handlers) throws IOException {
+        UUID nodeId = checkFileName(file);
+
+        ByteBuffer buf = allocateDirect(READ_BUFFER_SIZE).order(nativeOrder());
+
+        PerformanceStatisticsDeserializer des = new PerformanceStatisticsDeserializer(nodeId, handlers);
+
+        try (FileIO io = ioFactory.create(file)) {
+            while (true) {
+                int read = io.read(buf);
+
+                buf.flip();
+
+                if (read <= 0)

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450686877



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.EnumSet;
+
+/**
+ * Operation type.
+ */
+public enum OperationType {
+    /** Cache get. */
+    CACHE_GET,
+
+    /** Cache put. */
+    CACHE_PUT,
+
+    /** Cache remove. */
+    CACHE_REMOVE,
+
+    /** Cache get and put. */
+    CACHE_GET_AND_PUT,
+
+    /** Cache get and remove. */
+    CACHE_GET_AND_REMOVE,
+
+    /** Cache invoke. */
+    CACHE_INVOKE,
+
+    /** Cache lock. */
+    CACHE_LOCK,
+
+    /** Cache get all. */
+    CACHE_GET_ALL,
+
+    /** Cache put all. */
+    CACHE_PUT_ALL,
+
+    /** Cache remove all. */
+    CACHE_REMOVE_ALL,
+
+    /** Cache invoke all. */
+    CACHE_INVOKE_ALL,
+
+    /** Transaction commit. */
+    TX_COMMIT,
+
+    /** Transaction rollback. */
+    TX_ROLLBACK,
+
+    /** Query. */
+    QUERY,
+
+    /** Query reads. */
+    QUERY_READS,
+
+    /** Task. */
+    TASK,
+
+    /** Job. */
+    JOB;
+
+    /** Cache operations. */
+    public static final EnumSet<OperationType> CACHE_OPS = EnumSet.range(CACHE_GET, CACHE_INVOKE_ALL);

Review comment:
       Let's use explicit list of the operation instead of `range` here.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454926452



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Ignite. */
+    private static IgniteEx ignite;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        log = new ListeningTestLogger(GridAbstractTest.log);
+
+        ignite = startGrid(0);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            ignite.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);
+
+                tasks.incrementAndGet();
+
+                assertEquals(ignite.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);
+                assertTrue(duration >= 0);
+                assertEquals(-1, affPartId);
+            }
+
+            @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453690578



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";

Review comment:
       Other components use `_` as a separator and more compact naming such as `cp`, `binary_meta`.
   Let's use `perf_stat` name here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449513419



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);

Review comment:
       We shouldn't remove gathered statistics on each restart.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453701024



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";

Review comment:
       Should we make directory name more shortly 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446021135



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {
+        return writer;
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean statisticsEnabled() {
+        return writer.performanceStatisticsEnabled();
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @return Future to be completed on collecting started.
+     */
+    public IgniteInternalFuture<Void> startStatistics() {
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS)) {
+            return new GridFinishedFuture<>(
+                new IllegalStateException("Not all nodes in the cluster support collecting performance statistics."));
+        }
+
+        GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
+
+        UUID uuid = UUID.randomUUID();
+
+        synchronized (mux) {
+            if (disconnected || stopped) {
+                return new GridFinishedFuture<>(
+                    new IgniteFutureCancelledException("Node " + (stopped ? "stopped" : "disconnected")));
+            }
+
+            reqFuts.put(uuid, fut);
+        }
+
+        proc.start(uuid, true);
+
+        return fut;
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @return Future to be completed on collecting stopped.
+     */
+    public IgniteInternalFuture<Void> stopStatistics() {

Review comment:
       Typo: let's rename it to `stopCollectStatistics()`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449512213



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {

Review comment:
       This method can be private.
   
   

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {

Review comment:
       This method can be private.
   
   




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443616748



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -3324,6 +3418,9 @@ protected void removeAll0(final Collection<? extends K> keys) throws IgniteCheck
         if (statsEnabled)
             fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
 
+        if (profilingEnabled)
+            fut.listen(new ProfileClosure<>(CacheOperationType.REMOVE_ALL, start));

Review comment:
       I think we should push down methods `startTime` and `duration` from `GridCacheFuture` to `IgniteInternalFuture` therefore we can use the single non-static inner class instance to gather statistics both for cache statistics and performance statistics.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446036741



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {

Review comment:
       Let's encapsulate writer and introduce methods of this interface directly in the processor:
   We may want to filter out some event later or have several writers, etc.
   These changes should go directly in the processor and don't bother writer or core code.
   
   
   
   ```
                       ctx.performanceStatistics().task(
                           ses.getId(),
                           ses.getTaskName(),
                           ses.getStartTime(),
                           U.currentTimeMillis() - ses.getStartTime(),
                           worker.affPartId());
   ```




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450003883



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {

Review comment:
       Yes, existing `cancel` method can be used




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450788884



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.cancel();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.

Review comment:
       Let's remove those 3 parameters, because we only use constants for it, for 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.

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



[GitHub] [ignite] agoncharuk commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
agoncharuk commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r436546495



##########
File path: modules/profiling/src/main/java/org/apache/ignite/internal/profiling/handlers/CacheOperationsHandler.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.profiling.handlers;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.node.ArrayNode;
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static org.apache.ignite.internal.profiling.ProfilingFilesParser.currentNodeId;
+import static org.apache.ignite.internal.profiling.util.Utils.MAPPER;
+import static org.apache.ignite.internal.profiling.util.Utils.createArrayIfAbsent;
+import static org.apache.ignite.internal.profiling.util.Utils.createObjectIfAbsent;
+
+/**
+ * Builds JSON with aggregated cache operations statistics.
+ *
+ * Example:
+ * <pre>
+ * {
+ *    $nodeId : {
+ *       $cacheId : {
+ *          $opType : [ [ $startTime, $count] ]
+ *       }
+ *    }
+ * }
+ * </pre>
+ */
+public class CacheOperationsHandler implements IgniteProfilingHandler {
+    /** Field name of aggregated by caches/nodes values. */
+    private static final String TOTAL = "total";
+
+    /** Cache operations statistics: nodeId->cacheId->opType->aggregatedResults. */
+    private final Map<UUID, Map<Integer, Map<String, Map<Long, Integer>>>> res = new HashMap<>();

Review comment:
       Such nested maps are very hard to read, please introduce separate classes for internal maps.

##########
File path: modules/profiling/src/main/java/org/apache/ignite/internal/profiling/util/ProfilingDeserializer.java
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.profiling.util;
+
+import java.nio.ByteBuffer;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.profiling.IgniteProfiling;
+import org.apache.ignite.internal.profiling.IgniteProfiling.CacheOperationType;
+import org.apache.ignite.internal.profiling.LogFileProfiling;
+import org.apache.ignite.internal.profiling.LogFileProfiling.OperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static org.apache.ignite.internal.profiling.LogFileProfiling.readIgniteUuid;
+import static org.apache.ignite.internal.profiling.LogFileProfiling.readUuid;
+
+/**
+ * Profiling operations deserializer.
+ *
+ * @see LogFileProfiling
+ */
+public class ProfilingDeserializer implements AutoCloseable {
+    /** Cached strings by id. */
+    private final ConcurrentHashMap<Short, String> stringById = new ConcurrentHashMap<>();
+
+    /** Handlers to process deserialized operation. */
+    private final IgniteProfiling[] handlers;
+
+    /** @param handlers Handlers to process deserialized operation. */
+    public ProfilingDeserializer(IgniteProfiling... handlers) {
+        this.handlers = handlers;
+    }
+
+    /**
+     * Tries to deserialize profiling operation from buffer.
+     *
+     * @param buf Buffer.
+     * @return {@code True} if operation parsed. {@code False} if not enough bytes.
+     */
+    public boolean deserialize(ByteBuffer buf) {
+        int pos = buf.position();
+
+        if (buf.remaining() < 1)
+            return false;
+
+        byte opTypeByte = buf.get();
+
+        OperationType opType = OperationType.fromOrdinal(opTypeByte);
+
+        switch (opType) {
+            case CACHE_OPERATION: {
+                if (buf.remaining() < 1 + 4 + 8 + 8)

Review comment:
       Such calculations are present at least in LogFileProfiling too, need to extract them to a single place to avoid errors.

##########
File path: bin/profiling.sh
##########
@@ -0,0 +1,130 @@
+#!/usr/bin/env bash

Review comment:
       What is the reason of putting this into a separate file? There is an activity on the dev-list to extend control.sh to be able to support pluggable commands. I think it makes sense to include profiling control to control.sh instead of a separate file.

##########
File path: modules/profiling/src/main/java/org/apache/ignite/internal/profiling/util/ProfilingDeserializer.java
##########
@@ -0,0 +1,294 @@
+/*

Review comment:
       Need to add a test that validates serializing-deserializing of the operations. There is no need to start Ignite, just an instance of LogFileProfiling writing all possible operations and then an instance of ProfilingDeserializer reading those logged operations.

##########
File path: modules/profiling/report/index.html
##########
@@ -0,0 +1,188 @@
+<!DOCTYPE html>
+<!--
+ 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.
+-->
+<html lang="en">
+<head>
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <meta name="viewport" content="width=device-width, initial-scale=1, shrink-to-fit=no">
+    <meta name="theme-color" content="#563d7c">
+
+    <title>Ignite profiling report</title>
+
+    <!-- Bootstrap CSS. -->
+    <link rel="stylesheet" href="https://stackpath.bootstrapcdn.com/bootstrap/4.5.0/css/bootstrap.min.css"
+          integrity="sha384-9aIt2nRpC12Uk9gS9baDl411NQApFmC26EwAOH8WgZl5MYYxFfc+NcPb1dKGj7Sk"
+          crossorigin="anonymous">
+
+    <!-- Bootstrap table. -->
+    <link rel="stylesheet" href="https://unpkg.com/bootstrap-table@1.16.0/dist/bootstrap-table.min.css">
+
+    <!-- Bootstrap select. -->
+    <link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/bootstrap-select@1.13.14/dist/css/bootstrap-select.min.css">
+
+    <!-- Feather Icons. -->
+    <link rel="stylesheet" href="https://use.fontawesome.com/releases/v5.6.3/css/all.css"
+          integrity="sha384-UHRtZLI+pbxtHCWp1t77Bi1L4ZtiqrqD80Kn4Z8NTSRyMA2Fd33n5dQ8lWUE00s/"
+          crossorigin="anonymous">

Review comment:
       This relates to .js as well. 
   The report will not work without internet connection. We should consider either dropping external dependencies or packing them in tool resources so it can generate a standalone self-sufficient report.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453863935



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERF_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERF_STAT_DIR = "perf_stat";
+
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** File writer thread name. */
+    static final String WRITER_THREAD_NAME = "performance-statistics-writer";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics file I/O. */
+    private final FileIO fileIo;
+
+    /** Performance statistics file writer worker. */
+    private final FileWriter fileWriter;
+
+    /** File writer thread started flag. */
+    private boolean started;
+
+    /** File write buffer. */
+    private final SegmentedRingByteBuffer ringByteBuf;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuf = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) throws IgniteCheckedException, IOException {
+        log = ctx.log(getClass());
+
+        File file = statisticsFile(ctx);
+
+        U.delete(file);
+
+        fileIo = fileIoFactory.create(file);
+
+        log.info("Performance statistics file created [file=" + file.getAbsolutePath() + ']');
+
+        ringByteBuf = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+            SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+        fileWriter = new FileWriter(ctx, log);
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        assert !started;
+
+        new IgniteThread(fileWriter).start();
+
+        started = true;
+    }
+
+    /** Stops collecting performance statistics. */
+    public synchronized void stop() {
+        assert started;
+
+        // Stop accepting new records.
+        ringByteBuf.close();
+
+        U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+        // Make sure that all producers released their buffers to safe deallocate memory (in case of worker
+        // stopped abnormally).
+        ringByteBuf.poll();
+
+        ringByteBuf.free();
+
+        U.closeQuiet(fileIo);
+
+        started = false;
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        byte[] textBytes = text.getBytes();
+
+        doWrite(QUERY,
+            () -> 1 + 4 + textBytes.length + 4 + 8 + 8 + 8 + 1,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(textBytes.length);
+                buf.put(textBytes);
+                buf.putLong(id);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(success ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        byte[] nameBytes = taskName.getBytes();
+
+        doWrite(TASK,
+            () -> 24 + 4 + nameBytes.length + 8 + 8 + 4,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putInt(nameBytes.length);
+                buf.put(nameBytes);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.putInt(affPartId);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringByteBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true))
+                log.warning("The performance statistics file maximum size is reached.");
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());

Review comment:
       We should introduce an explicit index here.
   Otherwise, any edit in the middle of `OperationType` will broke deserialization.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454918314



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TestHandler.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Test performance statistics handler.
+ */
+public class TestHandler implements PerformanceStatisticsHandler {
+    @Override public void cacheOperation(UUID nodeId, OperationType type, int cacheId, long startTime, long duration) {
+        // No-op.
+    }
+
+    @Override public void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration,

Review comment:
       Typo: javadoc required.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443578928



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -6794,6 +6899,55 @@ public InvokeAllTimeStatClosure(CacheMetricsImpl metrics, final long start) {
         }
     }
 
+    /** */
+    private class ProfileClosure<T> implements CI1<IgniteInternalFuture<T>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Operation type. */
+        private final CacheOperationType op;
+
+        /** Start time in nanoseconds. */
+        private final long start;
+
+        /**
+         * @param op Operation type.
+         * @param start Start time in nanoseconds.
+         */
+        public ProfileClosure(CacheOperationType op, long start) {
+            this.op = op;
+            this.start = start;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteInternalFuture<T> fut) {
+            try {
+                if (!fut.isCancelled()) {
+                    fut.get();

Review comment:
       Why do we need `fut.get()` here?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451999682



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;

Review comment:
       I can't see why we should cache this variable here?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446131521



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatistics.java
##########
@@ -0,0 +1,633 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatistics implements IgnitePerformanceStatistics {

Review comment:
       Let's rename this to `FilePerformanceStatisticsWriter`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449203960



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.processors.performancestatistics;
+
+/**
+ * Operation type.
+ */
+public enum OperationType {
+    /** Cache operation. */
+    CACHE_OPERATION,

Review comment:
       Why we distinguish `CacheOperation` and `OperationType`?
   It seems to me that this two enums can be combiner into one pretty natural.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443684509



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
##########
@@ -2248,6 +2248,15 @@ private void onCacheStarted(GridCacheContext cacheCtx) throws IgniteCheckedExcep
                 ", mvcc=" + cacheCtx.mvccEnabled() + ']');
         }
 
+        if (ctx.metric().profilingEnabled()) {

Review comment:
       I don't think we need to collect performance statistics about cache start.
   It a rare operation that should be analyzed via log files.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450893043



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,375 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;

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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448876849



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Contains classes for Ignite performance statistics.
+ */
+
+package org.apache.ignite.internal.processors.performancestatistics;

Review comment:
       Ok. Removed.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Ignite performance statistics tests.
+ */
+
+package org.apache.ignite.internal.performancestatistics;

Review comment:
       Removed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449403938



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;

Review comment:
       Let's use `IgniteUtils#MB` here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448856696



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration, boolean commit);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime, long duration,
+        boolean success);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads,
+        long physicalReads);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut);
+
+    /** Cache operations types. */
+    public enum CacheOperationType {

Review comment:
       Let's rename this to `CacheOperation`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453726335



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics file writer worker. */
+    private final FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private final FileIO fileIo;
+
+    /** File write buffer. */
+    private final SegmentedRingByteBuffer ringByteBuffer;

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449564002



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                // Required to start writing data to the file.
+                notify();
+            }
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {

Review comment:
       This method used only once. Let's inline 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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r439994573



##########
File path: bin/profiling.sh
##########
@@ -0,0 +1,130 @@
+#!/usr/bin/env bash

Review comment:
       I have excluded the script from this PR. I move it and tool to create the report to the Ignite-extensions. 
   The script does not need a cluster. Profiling files can be parsed locally out of a cluster. It not assume to run a grid client node. This is why I use the separate script.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r487753361



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java
##########
@@ -0,0 +1,169 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.lang.management.ThreadInfo;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.mxbean.PerformanceStatisticsMBean;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.PERF_STAT_DIR;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.WRITER_THREAD_NAME;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Ignite performance statistics abstract test.
+ */
+public abstract class AbstractPerformanceStatisticsTest extends GridCommonAbstractTest {
+    /** */
+    public static final long TIMEOUT = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+
+        U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, true);
+    }
+
+    /** Starts collecting performance statistics. */
+    protected static void startCollectStatistics() throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        statisticsMBean(grids.get(0).name()).start();
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** Stops and reads collecting performance statistics. */
+    protected static void stopCollectStatisticsAndRead(TestHandler... handlers) throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        statisticsMBean(grids.get(0).name()).stop();
+
+        waitForStatisticsEnabled(false);
+
+        File dir = U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false);
+
+        new FilePerformanceStatisticsReader(handlers).read(Collections.singletonList(dir));
+    }
+
+    /** Wait for statistics started/stopped in the cluster. */
+    protected static void waitForStatisticsEnabled(boolean performanceStatsEnabled) throws Exception {
+        assertTrue(waitForCondition(() -> {
+            List<Ignite> grids = G.allGrids();
+
+            for (Ignite grid : grids)
+                if (performanceStatsEnabled != statisticsMBean(grid.name()).started())
+                    return false;
+
+            // Make sure that writer flushed data and stopped.
+            if (!performanceStatsEnabled) {
+                for (long id : U.getThreadMx().getAllThreadIds()) {
+                    ThreadInfo info = U.getThreadMx().getThreadInfo(id);
+
+                    if (info != null && info.getThreadState() != Thread.State.TERMINATED &&
+                        info.getThreadName().startsWith(WRITER_THREAD_NAME))
+                        return false;
+                }
+            }
+
+            return true;
+        }, TIMEOUT));
+    }
+
+    /**
+     * @param igniteInstanceName Ignite instance name.
+     * @return Ignite performance statistics MBean.
+     */
+    protected static PerformanceStatisticsMBean statisticsMBean(String igniteInstanceName) {
+        return getMxBean(igniteInstanceName, "PerformanceStatistics", PerformanceStatisticsMBeanImpl.class,
+            PerformanceStatisticsMBean.class);
+    }
+
+    /** Test performance statistics handler. */
+    public class TestHandler implements PerformanceStatisticsHandler {
+        /** {@inheritDoc} */
+        @Override public void cacheOperation(UUID nodeId, OperationType type, int cacheId, long startTime, long duration) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration,
+            boolean commited) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime,
+            long duration, boolean success) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id,
+            long logicalReads, long physicalReads) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+            int affPartId) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {
+            // No-op.
+        }
+    }
+
+    /** Node to run load from. */
+    enum LoadNode {

Review comment:
       Renamed




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448971399



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)

Review comment:
       Can we rewrite flow a bit to make this code more readable.
   ```
   if (fileWriter == null)
       return new GridFinishedFuture<>();
   
   return fileWriter.shutdown();
   ```




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453886828



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERF_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERF_STAT_DIR = "perf_stat";
+
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** File writer thread name. */
+    static final String WRITER_THREAD_NAME = "performance-statistics-writer";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics file I/O. */
+    private final FileIO fileIo;
+
+    /** Performance statistics file writer worker. */
+    private final FileWriter fileWriter;
+
+    /** File writer thread started flag. */
+    private boolean started;
+
+    /** File write buffer. */
+    private final SegmentedRingByteBuffer ringByteBuf;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuf = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) throws IgniteCheckedException, IOException {
+        log = ctx.log(getClass());
+
+        File file = statisticsFile(ctx);
+
+        U.delete(file);
+
+        fileIo = fileIoFactory.create(file);
+
+        log.info("Performance statistics file created [file=" + file.getAbsolutePath() + ']');
+
+        ringByteBuf = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+            SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+        fileWriter = new FileWriter(ctx, log);
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        assert !started;
+
+        new IgniteThread(fileWriter).start();
+
+        started = true;
+    }
+
+    /** Stops collecting performance statistics. */
+    public synchronized void stop() {
+        assert started;
+
+        // Stop accepting new records.
+        ringByteBuf.close();
+
+        U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+        // Make sure that all producers released their buffers to safe deallocate memory (in case of worker
+        // stopped abnormally).
+        ringByteBuf.poll();
+
+        ringByteBuf.free();
+
+        U.closeQuiet(fileIo);
+
+        started = false;
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        byte[] textBytes = text.getBytes();
+
+        doWrite(QUERY,
+            () -> 1 + 4 + textBytes.length + 4 + 8 + 8 + 8 + 1,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(textBytes.length);
+                buf.put(textBytes);
+                buf.putLong(id);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(success ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        byte[] nameBytes = taskName.getBytes();
+
+        doWrite(TASK,
+            () -> 24 + 4 + nameBytes.length + 8 + 8 + 4,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putInt(nameBytes.length);
+                buf.put(nameBytes);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.putInt(affPartId);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringByteBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true))
+                log.warning("The performance statistics file maximum size is reached.");
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r447047205



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter implements IgnitePerformanceStatistics {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatisticsWriter writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatisticsWriter(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {

Review comment:
       We can use `DistributedBooleanProperty` approach.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449008410



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)

Review comment:
       Can we use the string hash code as an identifier?
   Storing `ConcurrentHashMap` with the query string will create GC pressure if the user executes many different queries.
   
   We can use Set<Integer> or even BloomFilter implementation to check if some string already seen in the current statistics.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446065630



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatistics.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface IgnitePerformanceStatistics {

Review comment:
       We have `IgnitePerformanceStatistics` and its implementation `FilePerformanceStatistics`.
   We have implementation `FilePerformanceStatisticsWalker` but don't have a corresponding interface.
   
   I think we can remove `IgnitePerformanceStatistics` for now and introduce it when we will have two implementations.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r447025687



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatistics.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface IgnitePerformanceStatistics {

Review comment:
       Let's rename this to `PerformanceStatisticsHandler`.
   Also, this is part of the public API, right?
   If yes, then
   1. it should be located in `org.apache.ignite.performancestatistics`.
   2. let's mark this with the @IgniteExperimental.
   




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448992683



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;

Review comment:
       This is to make sure that the previous writer is stopped before starting a new one. I have removed it cause it seems unnecessary. 




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451520358



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            // Stop write new data.
+            if (buf != null)
+                buf.close();
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            if (buf != null)
+                buf.free();
+
+            U.closeQuiet(fileIo);
+
+            readyForFlushSize.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer ringBuf = ringByteBuffer;
+
+        // Starting.
+        if (ringBuf == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true)) {
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+
+        int readySize = readyForFlushSize.addAndGet(size);

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446020990



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {
+        return writer;
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean statisticsEnabled() {
+        return writer.performanceStatisticsEnabled();
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @return Future to be completed on collecting started.
+     */
+    public IgniteInternalFuture<Void> startStatistics() {

Review comment:
       Typo: let's rename it to `startCollectStatistics()`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448864687



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsHandler.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperationType type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commit) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commit ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)
+                return -1;
+
+            return stringIds.computeIfAbsent(str,
+                s -> (short)idsGen.updateAndGet(id -> id < MAX_CACHED_STRING_COUNT ? id + 1 : -1));
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private IgniteInternalFuture<Void> shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                notify();
+            }
+
+            return stopFut;
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+        }
+
+        /** Logs warning message and stops collecting statistics. */
+        void onMaxFileSizeReached() {
+            if (stopByMaxSize.compareAndSet(false, true)) {
+                fileWriter.shutdown();
+
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+        }
+
+        /** Stops collecting statistics. */
+        void stopStatistics() {
+            try {
+                ctx.performanceStatistics().stopCollectStatistics();
+            }
+            catch (IgniteCheckedException e) {
+                log.error("Failed to stop performance statistics.", e);
+            }
+        }
+    }
+
+    /** Operation type. */
+    public enum OperationType {

Review comment:
       Let's make this enum not internal. 
   




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449399960



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return {@code True} if string hash code is cached. {@code False} if need write string.  */
+        boolean stringCached(String str) {
+            boolean cached = cachedStrings.contains(str.hashCode());
+
+            if (!cached)
+                cachedStrings.add(str.hashCode());
+
+            return cached;
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {

Review comment:
       This method used only once. Let's inline 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.

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



[GitHub] [ignite] nizhikov commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-689658459


   We need to add tests that execute operations from client node or thin clients across the cluster of at least 2 nodes.
   And check we can observe operations results on all nodes.


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446136217



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMBean.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.mxbean.MXBeanDescription;
+
+/**
+ * MBean that provides access to performance statistics management.
+ */
+@MXBeanDescription("MBean provide access to performance statistics management.")
+public interface IgnitePerformanceStatisticsMBean {

Review comment:
       This interface should be in `org.apache.ignite.mxbean`.
   Also, let's mark all public API with the @IgniteExperimental.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449403479



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;

Review comment:
       Let's use `IgniteUtils#GB` here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449400393



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return {@code True} if string hash code is cached. {@code False} if need write string.  */
+        boolean stringCached(String str) {
+            boolean cached = cachedStrings.contains(str.hashCode());
+
+            if (!cached)
+                cachedStrings.add(str.hashCode());
+
+            return cached;
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        notify();

Review comment:
       Can you please, add a comment to clarify that this `notify` required to start writing data to the file.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446137472



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMbeanImpl.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * {@link IgnitePerformanceStatisticsMBean} implementation.
+ */
+public class IgnitePerformanceStatisticsMbeanImpl implements IgnitePerformanceStatisticsMBean {

Review comment:
       Typo: MbeanImpl -> M*B*eanImpl




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449207778



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(text.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(taskName.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();

Review comment:
       Why we ignore `poll` results here?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453686063



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");
+            }
+        }
+        catch (Exception e) {
+            log.error("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops performance statistics writer. */
+    private void stopWriter() {
+        synchronized (this) {

Review comment:
       Let's have explicit `mux` Object for synchronization to avoid contention if some external object will sync of processor.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451510887



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            // Stop write new data.
+            if (buf != null)
+                buf.close();
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            if (buf != null)
+                buf.free();
+
+            U.closeQuiet(fileIo);
+
+            readyForFlushSize.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer ringBuf = ringByteBuffer;
+
+        // Starting.
+        if (ringBuf == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true)) {
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+
+        int readySize = readyForFlushSize.addAndGet(size);
+
+        if (readySize >= DFLT_FLUSH_SIZE)
+            fileWriter.wakeUp();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Stops collecting statistics in the cluster. */
+    void stopStatistics() {
+        try {
+            ctx.performanceStatistics().stopCollectStatistics();
+        }
+        catch (IgniteCheckedException e) {
+            log.error("Failed to stop performance statistics.", e);
+        }
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /**
+         * @param ctx Kernal context.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log,
+                ctx.workersRegistry());
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < DFLT_FLUSH_SIZE && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (SegmentedRingByteBuffer.ReadSegment seg : segs) {
+                    updateHeartbeat();
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                stopStatistics();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454920083



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TopologyChangesTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.junit.Test;
+
+/**
+ * Tests topology changes during collecting performance statistics.
+ */
+public class TopologyChangesTest extends AbstractPerformanceStatisticsTest {
+    /** */
+    private boolean persistence;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                )
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeJoin() throws Exception {
+        startGrid(0);
+
+        startCollectStatistics();
+
+        startGrid(1);
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterRestartWithPersistence() throws Exception {
+        persistence = true;

Review comment:
       We should clear `persistence` value in the finally block.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448877348



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter implements IgnitePerformanceStatistics {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatisticsWriter writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatisticsWriter(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {

Review comment:
       I have mplemented this as 'DistributedMetaStorage' listener.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446089022



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.

Review comment:
       We should support the case when `file` represents a directory




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r452003435



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;

Review comment:
       For example, if `fileIo = fileIoFactory.create(file);` throws an exception and the writer will not be created




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446018295



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -2588,6 +2630,9 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter)
         if (statsEnabled && stored)
             metrics0().addPutTimeNanos(System.nanoTime() - start);
 
+        if (performanceStatsEnabled)

Review comment:
       Typo: let's use `performanceStatsEnabled` name here and in all other cases.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446113191



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
##########
@@ -2020,6 +2020,14 @@ else if (removeFromActive(jobWorker)) {
                         rwLock.readUnlock();
                     }
                 }
+
+                if (ctx.performanceStatistics().enabled()) {

Review comment:
       Why we do this inside `if (!worker.isInternal()) {`?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451486882



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            // Stop write new data.
+            if (buf != null)
+                buf.close();
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            if (buf != null)
+                buf.free();
+
+            U.closeQuiet(fileIo);
+
+            readyForFlushSize.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer ringBuf = ringByteBuffer;
+
+        // Starting.
+        if (ringBuf == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true)) {
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+
+        int readySize = readyForFlushSize.addAndGet(size);

Review comment:
       I think we should do the following:
   
   1. Each time `readyForFlushSize` cross `DFLT_FLUSH_SIZE` we should clear it to zero.
   2. Each time we see zero as a result of `updateAndGet` we should notify `FileWriter` to be sure it start flushing.
   3. `FileWriter` just flush all the data it have for the moment(slightly more then `DFLT_FLUSH_SIZE` because of concurrent writes.
   
   We shouldn't touch `readyForFlushSize` in the `FileWriter`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448980370



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();

Review comment:
       Why `flushBuffer` is in blocking section?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448859666



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.

Review comment:
       This flag should be named `committed`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454923058



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TopologyChangesTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.junit.Test;
+
+/**
+ * Tests topology changes during collecting performance statistics.
+ */
+public class TopologyChangesTest extends AbstractPerformanceStatisticsTest {
+    /** */
+    private boolean persistence;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                )
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeJoin() throws Exception {
+        startGrid(0);
+
+        startCollectStatistics();
+
+        startGrid(1);
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterRestartWithPersistence() throws Exception {
+        persistence = true;

Review comment:
       No, it is unnecessary. JUnit creates a new instance for 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.

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



[GitHub] [ignite] nizhikov commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-658667712


   Do we have test for a `SCAN` query?


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446077333



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void walkFile(Path file, IgnitePerformanceStatistics... handlers) throws IOException {

Review comment:
       Let's rename this to `read`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446065630



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatistics.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface IgnitePerformanceStatistics {

Review comment:
       We have `IgnitePerformanceStatistics` and its implementation `FilePerformanceStatistics`.
   We have implementation `FilePerformanceStatisticsWalker` but don't have a corresponding interface.
   
   I think we can remove `IgnitePerformanceStatistics` for now and introduce it when we will have two implementations.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446046037



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -2588,6 +2630,9 @@ public boolean put(final K key, final V val, final CacheEntryPredicate filter)
         if (statsEnabled && stored)
             metrics0().addPutTimeNanos(System.nanoTime() - start);
 
+        if (performanceStatsEnabled)

Review comment:
       Fixed.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449230984



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.processors.performancestatistics;
+
+/**
+ * Operation type.
+ */
+public enum OperationType {
+    /** Cache operation. */
+    CACHE_OPERATION,

Review comment:
       IMHO, it'll be strange to provide method to read cache operation with **not-only-cache-operation** type:
   `void cacheOperation(UUID nodeId, OperationType type, int cacheId, long startTime, long duration);`




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

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



[GitHub] [ignite] NSAmelchev commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-658669673


   > Do we have test for a `SCAN` query?
   
   Yes. See PerformanceStatisticsQueryTest#testScanQuery()


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454918147



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TestHandler.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Test performance statistics handler.
+ */
+public class TestHandler implements PerformanceStatisticsHandler {

Review comment:
       Let's make this class internal in the `AbstractPerformanceStatisticsTest`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449511785



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {

Review comment:
       This method can be private.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446092742



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void walkFile(Path file, IgnitePerformanceStatistics... handlers) throws IOException {
+        ByteBuffer buf = allocateDirect(READ_BUFFER_SIZE).order(nativeOrder());
+
+        try (
+            FileIO io = ioFactory.create(file.toFile());
+            PerformanceStatisticsDeserializer des = new PerformanceStatisticsDeserializer(handlers)
+        ) {
+            while (true) {
+                int read = io.read(buf);
+
+                buf.flip();
+
+                if (read <= 0)
+                    break;
+
+                while (true) {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void walkFile(Path file, IgnitePerformanceStatistics... handlers) throws IOException {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451484424



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);

Review comment:
       The previous file can be closed during writing records. So the last record can be written not fully. It may lead to the case when the file can not be read.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446125037



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
##########
@@ -2020,6 +2020,14 @@ else if (removeFromActive(jobWorker)) {
                         rwLock.readUnlock();
                     }
                 }
+
+                if (ctx.performanceStatistics().enabled()) {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r452000738



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            if (buf != null) {
+                buf.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                buf.poll();
+
+                buf.free();
+            }
+
+            U.closeQuiet(fileIo);
+
+            writtenToBuffer.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;

Review comment:
       I can't see why we should cache this variable.
   We also, should check `enabled` flag here, but assign it to `true` as a last statement in the `start`.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450703009



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/OperationType.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.EnumSet;
+
+/**
+ * Operation type.
+ */
+public enum OperationType {
+    /** Cache get. */
+    CACHE_GET,
+
+    /** Cache put. */
+    CACHE_PUT,
+
+    /** Cache remove. */
+    CACHE_REMOVE,
+
+    /** Cache get and put. */
+    CACHE_GET_AND_PUT,
+
+    /** Cache get and remove. */
+    CACHE_GET_AND_REMOVE,
+
+    /** Cache invoke. */
+    CACHE_INVOKE,
+
+    /** Cache lock. */
+    CACHE_LOCK,
+
+    /** Cache get all. */
+    CACHE_GET_ALL,
+
+    /** Cache put all. */
+    CACHE_PUT_ALL,
+
+    /** Cache remove all. */
+    CACHE_REMOVE_ALL,
+
+    /** Cache invoke all. */
+    CACHE_INVOKE_ALL,
+
+    /** Transaction commit. */
+    TX_COMMIT,
+
+    /** Transaction rollback. */
+    TX_ROLLBACK,
+
+    /** Query. */
+    QUERY,
+
+    /** Query reads. */
+    QUERY_READS,
+
+    /** Task. */
+    TASK,
+
+    /** Job. */
+    JOB;
+
+    /** Cache operations. */
+    public static final EnumSet<OperationType> CACHE_OPS = EnumSet.range(CACHE_GET, CACHE_INVOKE_ALL);

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454920618



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TopologyChangesTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.junit.Test;
+
+/**
+ * Tests topology changes during collecting performance statistics.
+ */
+public class TopologyChangesTest extends AbstractPerformanceStatisticsTest {
+    /** */
+    private boolean persistence;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                )
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeJoin() throws Exception {
+        startGrid(0);
+
+        startCollectStatistics();
+
+        startGrid(1);
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterRestartWithPersistence() throws Exception {
+        persistence = true;

Review comment:
       Can we execute all tests both in the persinstence and non persistence mode?
   We can use `Parametrized` test for this.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TopologyChangesTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.junit.Test;
+
+/**
+ * Tests topology changes during collecting performance statistics.
+ */
+public class TopologyChangesTest extends AbstractPerformanceStatisticsTest {
+    /** */
+    private boolean persistence;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                )
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeJoin() throws Exception {
+        startGrid(0);
+
+        startCollectStatistics();
+
+        startGrid(1);
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterRestartWithPersistence() throws Exception {
+        persistence = true;

Review comment:
       Can we execute all tests both in the persistence and non-persistence mode?
   We can use `Parametrized` test for this.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454976983



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TopologyChangesTest.java
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.junit.Test;
+
+/**
+ * Tests topology changes during collecting performance statistics.
+ */
+public class TopologyChangesTest extends AbstractPerformanceStatisticsTest {
+    /** */
+    private boolean persistence;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setDataStorageConfiguration(
+            new DataStorageConfiguration()
+                .setDefaultDataRegionConfiguration(
+                    new DataRegionConfiguration().setPersistenceEnabled(persistence)
+                )
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testNodeJoin() throws Exception {
+        startGrid(0);
+
+        startCollectStatistics();
+
+        startGrid(1);
+
+        waitForStatisticsEnabled(true);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testClusterRestartWithPersistence() throws Exception {
+        persistence = true;

Review comment:
       I have used `Parametrized` for 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449192874



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);

Review comment:
       Let's introduce the following method to generalize write:
   
   ```
   public void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
           FileWriter writer = fileWriter;
   
           if (writer == null)
               return;
   
           int size = sizeSupplier.apply();
           SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(op, size);
   
           if (seg == null)
               return;
   
           writer.apply(seg.buffer());
           seg.release();
   }
   ```
   
   This will allow us to implement only size calculation and write to the `ByteBuffer` logic for specific operations.




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

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



[GitHub] [ignite] nizhikov commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-640462056


   Hello! Thanks for the PR.
   Let's make the following changes:
   
   1. core module should contain all implementations and interfaces that required to:
     a. Collect profiling data.
     b. Iterate through profiling data from disk
   2. html report generation should be in ignite-extension as a separate module.
   3. please, create the ticket to provide ability to profile only specific operations. Something like `profiling.enable(TRANSACTION, CACHE_OPERATIONS)`


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451999890



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            if (buf != null) {
+                buf.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                buf.poll();
+
+                buf.free();
+            }
+
+            U.closeQuiet(fileIo);
+
+            writtenToBuffer.set(0);

Review comment:
       It seems we should set up default values in the `start` method.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451139193



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                // Required to start writing data to the file.
+                notify();
+            }
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                // Required to start writing data to the file.
+                notify();
+            }
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+        }
+
+        /** Logs warning message and stops collecting statistics. */
+        void onMaxFileSizeReached() {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.cancel();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.cancel();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448982212



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {

Review comment:
       Why do we need to check `Thread.interrupted` here?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454307949



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.jobRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.queryReadsRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.queryRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.taskRecordSize;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionRecordSize;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = (int)(8 * U.MB);
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN =
+        "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /** File name pattern. */
+    private static final Pattern FILE_PATTERN = Pattern.compile("^node-(" + UUID_STR_PATTERN + ").prf$");
+
+    /** IO factory. */
+    private final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /** Handlers to process deserialized operations. */
+    private final PerformanceStatisticsHandler[] handlers;
+
+    /** @param handlers Handlers to process deserialized operations. */
+    FilePerformanceStatisticsReader(PerformanceStatisticsHandler... handlers) {
+        A.ensure(handlers != null, "At least one handler expected.");

Review comment:
       We can use `A.notEmpty` here.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446006805



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -3324,6 +3418,9 @@ protected void removeAll0(final Collection<? extends K> keys) throws IgniteCheck
         if (statsEnabled)
             fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
 
+        if (profilingEnabled)
+            fut.listen(new ProfileClosure<>(CacheOperationType.REMOVE_ALL, start));

Review comment:
       > We should have separate class for each operations so we can eliminate `operation` variable from `ProfileClosure`.
   
   I have used lambda expressions.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448868444



##########
File path: modules/core/src/main/java/org/apache/ignite/mxbean/PerformanceStatisticsMBean.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.mxbean;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * MBean that provides access to performance statistics management.
+ */
+@IgniteExperimental
+@MXBeanDescription("MBean provide access to performance statistics management.")
+public interface PerformanceStatisticsMBean {
+    /**
+     * Start collecting performance statistics in the cluster with default settings.
+     *
+     * @see FilePerformanceStatisticsWriter#DFLT_FILE_MAX_SIZE
+     * @see FilePerformanceStatisticsWriter#DFLT_BUFFER_SIZE
+     * @see FilePerformanceStatisticsWriter#DFLT_FLUSH_SIZE
+     */
+    @MXBeanDescription("Start collecting performance statistics in the cluster.")
+    public void start() throws IgniteCheckedException;
+
+    /** Stop collecting performance statistics in the cluster. */
+    @MXBeanDescription("Stop collecting performance statistics in the cluster.")
+    public void stop() throws IgniteCheckedException;
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    @MXBeanDescription("True if collecting performance statistics is enabled.")
+    public boolean enabled();

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration, boolean commit);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    void query(UUID nodeId, GridCacheQueryType type, String text, long id, long startTime, long duration,
+        boolean success);
+
+    /**
+     * @param nodeId Node id.
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    void queryReads(UUID nodeId, GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads,
+        long physicalReads);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId);
+
+    /**
+     * @param nodeId Node id.
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut);
+
+    /** Cache operations types. */
+    public enum CacheOperationType {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsHandler.java
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * The interface represents performance statistics operations collection for purposes of troubleshooting and
+ * performance analysis.
+ */
+public interface PerformanceStatisticsHandler {
+    /**
+     * @param nodeId Node id.
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    void cacheOperation(UUID nodeId, CacheOperationType type, int cacheId, long startTime, long duration);
+
+    /**
+     * @param nodeId Node id.
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,690 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.PerformanceStatisticsHandler.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperationType type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commit {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commit) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commit ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)
+                return -1;
+
+            return stringIds.computeIfAbsent(str,
+                s -> (short)idsGen.updateAndGet(id -> id < MAX_CACHED_STRING_COUNT ? id + 1 : -1));
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private IgniteInternalFuture<Void> shutdown() {
+            isCancelled = true;
+
+            synchronized (this) {
+                notify();
+            }
+
+            return stopFut;
+        }
+
+        /** Logs warning message about small buffer size if not logged yet. */
+        void logSmallBufferMessage() {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+        }
+
+        /** Logs warning message and stops collecting statistics. */
+        void onMaxFileSizeReached() {
+            if (stopByMaxSize.compareAndSet(false, true)) {
+                fileWriter.shutdown();
+
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+        }
+
+        /** Stops collecting statistics. */
+        void stopStatistics() {
+            try {
+                ctx.performanceStatistics().stopCollectStatistics();
+            }
+            catch (IgniteCheckedException e) {
+                log.error("Failed to stop performance statistics.", e);
+            }
+        }
+    }
+
+    /** Operation type. */
+    public enum OperationType {

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449512131



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {

Review comment:
       This method can be private.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449129190



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {

Review comment:
       Let's rename this to `enabled`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449399754



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();

Review comment:
       It seems to me that all resource cleaning should be done in `finally` block to prevent resource leaking on any unhandled exception.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448850776



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Contains classes for Ignite performance statistics.
+ */
+
+package org.apache.ignite.internal.processors.performancestatistics;

Review comment:
       AFAIK we don't need package descriptions for internal packages.
   Let's remove this file.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r457918067



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsQueryTest.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.Query;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.junit.Test;
+
+import static org.apache.ignite.cluster.ClusterState.ACTIVE;
+import static org.apache.ignite.cluster.ClusterState.INACTIVE;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SCAN;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
+
+/** Tests query performance statistics. */
+public class PerformanceStatisticsQueryTest extends AbstractPerformanceStatisticsTest {
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;

Review comment:
       Page size added




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446019432



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.Serializable;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.distributed.DistributedProcess;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteFutureCancelledException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PERFORMANCE_STAT_PROC;
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.PERFORMANCE_STATISTICS;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting statistics.
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Process to start/stop statistics. */
+    private final DistributedProcess<Boolean, Boolean> proc;
+
+    /** Performance statistics writer. */
+    private final FilePerformanceStatistics writer;
+
+    /** Synchronization mutex for request futures. */
+    private final Object mux = new Object();
+
+    /** Enable/disable statistics request futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Void>> reqFuts = new ConcurrentHashMap<>();
+
+    /** Disconnected flag. */
+    private volatile boolean disconnected;
+
+    /** Stopped flag. */
+    private volatile boolean stopped;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        writer = new FilePerformanceStatistics(ctx);
+
+        proc = new DistributedProcess<>(ctx, PERFORMANCE_STATISTICS, start -> {
+            if (start) {
+                return ctx.closure().callLocalSafe(() -> {
+                    if (start)
+                        writer.start();
+
+                    return true;
+                });
+            }
+
+            return writer.stop().chain(f -> true);
+        }, (uuid, res, err) -> {
+            if (!F.isEmpty(err) && statisticsEnabled())
+                writer.stop();
+
+            synchronized (mux) {
+                GridFutureAdapter<Void> fut = reqFuts.get(uuid);
+
+                if (fut != null) {
+                    if (!F.isEmpty(err))
+                        fut.onDone(new IgniteException("Unable to process request [err=" + err + ']'));
+                    else
+                        fut.onDone();
+                }
+            }
+        });
+    }
+
+    /** @return Performance statistics writer. */
+    public IgnitePerformanceStatistics writer() {
+        return writer;
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean statisticsEnabled() {

Review comment:
       Typo: let's rename this to `enabled`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446007040



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
##########
@@ -234,6 +241,9 @@
     /** Nonheap memory metrics. */
     private final MemoryUsageMetrics nonHeap;
 
+    /** Profiling. */
+    private final FileProfiling profiling;

Review comment:
       I have moved code to the `PerformaceStatisticsProcessor`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446006634



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/profiling/IgniteProfilingMBean.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.profiling;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.mxbean.MXBeanDescription;
+import org.apache.ignite.mxbean.MXBeanParameter;
+
+/**
+ * MBean provide access to profiling management.
+ */
+@MXBeanDescription("MBean provide access to profiling management.")
+public interface IgniteProfilingMBean {
+    /**
+     * Start profiling in the cluster with default settings.
+     *
+     * @see FileProfiling#DFLT_FILE_MAX_SIZE
+     * @see FileProfiling#DFLT_BUFFER_SIZE
+     * @see FileProfiling#DFLT_FLUSH_SIZE
+     */
+    @MXBeanDescription("Start profiling in the cluster.")
+    public void startProfiling() throws IgniteCheckedException;

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448972046



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;

Review comment:
       The essence of this assert is not clear for me.
   




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448851136



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Ignite performance statistics tests.
+ */
+
+package org.apache.ignite.internal.performancestatistics;

Review comment:
       AFAIK we don't need package descriptions for internal packages.
   Let's remove this file.
   
   




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451474344



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);

Review comment:
       Let's append to the file if it exists.
   It seems all we need to do is to remove this line.
   
   We need a ticket to provide `clear` command, also.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446141492



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatistics.java
##########
@@ -0,0 +1,633 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatistics implements IgnitePerformanceStatistics {

Review comment:
       Done

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMBean.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.mxbean.MXBeanDescription;
+
+/**
+ * MBean that provides access to performance statistics management.
+ */
+@MXBeanDescription("MBean provide access to performance statistics management.")
+public interface IgnitePerformanceStatisticsMBean {

Review comment:
       Done

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMbeanImpl.java
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridKernalContext;
+
+/**
+ * {@link IgnitePerformanceStatisticsMBean} implementation.
+ */
+public class IgnitePerformanceStatisticsMbeanImpl implements IgnitePerformanceStatisticsMBean {

Review comment:
       Fixed




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r452001285



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,529 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;

Review comment:
       I can't see why we should cache this variable.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454926305



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TestHandler.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Test performance statistics handler.
+ */
+public class TestHandler implements PerformanceStatisticsHandler {

Review comment:
       Done.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/TestHandler.java
##########
@@ -0,0 +1,57 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteUuid;
+
+/**
+ * Test performance statistics handler.
+ */
+public class TestHandler implements PerformanceStatisticsHandler {
+    @Override public void cacheOperation(UUID nodeId, OperationType type, int cacheId, long startTime, long duration) {
+        // No-op.
+    }
+
+    @Override public void transaction(UUID nodeId, GridIntList cacheIds, long startTime, long duration,

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.

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



[GitHub] [ignite] nizhikov commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-647501884


   It seems to me that work "profiling" not clear enough.
   "Profiling" is a process that can be done with the data that will be collected with the patch.
   But, "profiling" is not a single use-case for the data.
   I can imagine something like automatic parameters tuning, affinity function tweaks, etc. based on the data about performance.
   
   I propose to use "Performance statistics" or similar words to describe the new feature and "performance report" to name report based on collected data.
   
   WDYT?


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448855638



##########
File path: modules/core/src/main/java/org/apache/ignite/mxbean/PerformanceStatisticsMBean.java
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.mxbean;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * MBean that provides access to performance statistics management.
+ */
+@IgniteExperimental
+@MXBeanDescription("MBean provide access to performance statistics management.")
+public interface PerformanceStatisticsMBean {
+    /**
+     * Start collecting performance statistics in the cluster with default settings.
+     *
+     * @see FilePerformanceStatisticsWriter#DFLT_FILE_MAX_SIZE
+     * @see FilePerformanceStatisticsWriter#DFLT_BUFFER_SIZE
+     * @see FilePerformanceStatisticsWriter#DFLT_FLUSH_SIZE
+     */
+    @MXBeanDescription("Start collecting performance statistics in the cluster.")
+    public void start() throws IgniteCheckedException;
+
+    /** Stop collecting performance statistics in the cluster. */
+    @MXBeanDescription("Stop collecting performance statistics in the cluster.")
+    public void stop() throws IgniteCheckedException;
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    @MXBeanDescription("True if collecting performance statistics is enabled.")
+    public boolean enabled();

Review comment:
       Let's rename this to `started`




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449227476



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter == null)
+            return new GridFinishedFuture<>();
+
+        return fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(text.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putInt(taskName.hashCode());
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Hashcodes of cached strings. */
+        private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();

Review comment:
       Replaced with `flushBuffer();`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449562489



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {

Review comment:
       Let's rename this to `stop`.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449526961



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448972046



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;

Review comment:
       The essence of this assert is not clear for me.
   Can you, please, clarify 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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r444386282



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -6794,6 +6899,55 @@ public InvokeAllTimeStatClosure(CacheMetricsImpl metrics, final long start) {
         }
     }
 
+    /** */
+    private class ProfileClosure<T> implements CI1<IgniteInternalFuture<T>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Operation type. */
+        private final CacheOperationType op;
+
+        /** Start time in nanoseconds. */
+        private final long start;
+
+        /**
+         * @param op Operation type.
+         * @param start Start time in nanoseconds.
+         */
+        public ProfileClosure(CacheOperationType op, long start) {
+            this.op = op;
+            this.start = start;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteInternalFuture<T> fut) {
+            try {
+                if (!fut.isCancelled()) {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r439997189



##########
File path: modules/profiling/report/index.html
##########
@@ -0,0 +1,188 @@
+<!DOCTYPE html>
+<!--
+ 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.
+-->
+<html lang="en">
+<head>
+    <meta http-equiv="Content-Type" content="text/html; charset=UTF-8">
+    <meta name="viewport" content="width=device-width, initial-scale=1, shrink-to-fit=no">
+    <meta name="theme-color" content="#563d7c">
+
+    <title>Ignite profiling report</title>
+
+    <!-- Bootstrap CSS. -->
+    <link rel="stylesheet" href="https://stackpath.bootstrapcdn.com/bootstrap/4.5.0/css/bootstrap.min.css"
+          integrity="sha384-9aIt2nRpC12Uk9gS9baDl411NQApFmC26EwAOH8WgZl5MYYxFfc+NcPb1dKGj7Sk"
+          crossorigin="anonymous">
+
+    <!-- Bootstrap table. -->
+    <link rel="stylesheet" href="https://unpkg.com/bootstrap-table@1.16.0/dist/bootstrap-table.min.css">
+
+    <!-- Bootstrap select. -->
+    <link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/bootstrap-select@1.13.14/dist/css/bootstrap-select.min.css">
+
+    <!-- Feather Icons. -->
+    <link rel="stylesheet" href="https://use.fontawesome.com/releases/v5.6.3/css/all.css"
+          integrity="sha384-UHRtZLI+pbxtHCWp1t77Bi1L4ZtiqrqD80Kn4Z8NTSRyMA2Fd33n5dQ8lWUE00s/"
+          crossorigin="anonymous">

Review comment:
       I have included all dependencies to the release package. Now it fully offline tool. See https://github.com/apache/ignite-extensions/pull/16




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448863136



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Contains classes for Ignite performance statistics.
+ */
+
+package org.apache.ignite.internal.processors.performancestatistics;

Review comment:
       I have added it cause wiki says to add info files despite internal it or no:
   https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-Creation
   https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-package-info.java




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454385234



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.lang.management.ThreadInfo;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.PERF_STAT_DIR;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.WRITER_THREAD_NAME;
+import static org.apache.ignite.internal.processors.performancestatistics.TestFilePerformanceStatisticsReader.readToLog;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Ignite performance statistics abstract test.
+ */
+public abstract class AbstractPerformanceStatisticsTest extends GridCommonAbstractTest {
+    /** */
+    public static final long TIMEOUT = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+
+        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false));
+    }
+
+    /** Starts collecting performance statistics. */
+    protected static void startCollectStatistics() throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        IgniteEx ignite = (IgniteEx)grids.get(0);
+
+        ignite.context().performanceStatistics().startCollectStatistics();

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446077134



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {

Review comment:
       Let's rename this to `FilePerformanceStatisticsReader`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453761049



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";

Review comment:
       This name is not a prefix, it a full name, isnt'it?
   Let's rename this to `PERFORMANCE_STAT_KEY`




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454180692



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = (int)(8 * U.MB);
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN =
+        "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /** File name pattern. */
+    private static final Pattern FILE_PATTERN = Pattern.compile("^node-(" + UUID_STR_PATTERN + ").prf$");
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics files.
+     *
+     * @param filesOrDirs Files or directories.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void read(List<File> filesOrDirs, PerformanceStatisticsHandler... handlers) throws IOException {
+        List<File> files = resolveFiles(filesOrDirs);
+
+        if (files.isEmpty())
+            return;
+
+        for (File file : files)
+            readFile(file, handlers);
+    }
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    private static void readFile(File file, PerformanceStatisticsHandler... handlers) throws IOException {
+        UUID nodeId = checkFileName(file);
+
+        ByteBuffer buf = allocateDirect(READ_BUFFER_SIZE).order(nativeOrder());
+
+        PerformanceStatisticsDeserializer des = new PerformanceStatisticsDeserializer(nodeId, handlers);
+
+        try (FileIO io = ioFactory.create(file)) {
+            while (true) {
+                int read = io.read(buf);
+
+                buf.flip();
+
+                if (read <= 0)

Review comment:
       Nit: We should firstly check read results and after flip the buffer. 




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446063535



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/metric/GridMetricManager.java
##########
@@ -58,8 +58,8 @@
 import org.apache.ignite.spi.metric.HistogramMetric;
 import org.apache.ignite.spi.metric.Metric;
 import org.apache.ignite.spi.metric.MetricExporterSpi;
-import org.apache.ignite.spi.metric.ReadOnlyMetricRegistry;

Review comment:
       Typo: plese, revert this change




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r451477406



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,517 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    @Nullable private volatile FileIO fileIo;
+
+    /** File write buffer. */
+    @Nullable private volatile SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Size of ready for flushing bytes. */
+    private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {
+        synchronized (this) {
+            if (enabled)
+                return;
+
+            enabled = true;
+
+            try {
+                File file = statisticsFile(ctx);
+
+                U.delete(file);
+
+                fileIo = fileIoFactory.create(file);
+
+                ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+                    SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+                ringByteBuffer.init(0);
+
+                fileWriter = new FileWriter(ctx, log);
+
+                new IgniteThread(fileWriter).start();
+
+                log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+            }
+            catch (IOException | IgniteCheckedException e) {
+                log.error("Failed to start performance statistics writer.", e);
+
+                stopStatistics();
+
+                throw new IgniteException("Failed to start performance statistics writer.", e);
+            }
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            FileWriter fileWriter = this.fileWriter;
+
+            SegmentedRingByteBuffer buf = ringByteBuffer;
+
+            // Stop write new data.
+            if (buf != null)
+                buf.close();
+
+            // Make sure that all buffer's producers released to safe deallocate memory.
+            if (fileWriter != null)
+                U.awaitForWorkersStop(Collections.singleton(fileWriter), true, log);
+
+            if (buf != null)
+                buf.free();
+
+            U.closeQuiet(fileIo);
+
+            readyForFlushSize.set(0);
+            smallBufLogged.set(false);
+            stopByMaxSize.set(false);
+            cachedStrings.clear();
+
+            log.info("Performance statistics writer stopped.");
+        }
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt() + /*type*/ 1;
+
+        SegmentedRingByteBuffer ringBuf = ringByteBuffer;
+
+        // Starting.
+        if (ringBuf == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = ringBuf.offer(size);
+
+        if (seg == null) {
+            if (smallBufLogged.compareAndSet(false, true)) {
+                log.warning("The performance statistics in-memory buffer size is too small. Some operations " +
+                    "will not be logged.");
+            }
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled() && stopByMaxSize.compareAndSet(false, true)) {
+                stopStatistics();
+
+                log.warning("The performance statistics file maximum size is reached. " +
+                    "Performance statistics collecting will be stopped.");
+            }
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+
+        int readySize = readyForFlushSize.addAndGet(size);
+
+        if (readySize >= DFLT_FLUSH_SIZE)
+            fileWriter.wakeUp();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Stops collecting statistics in the cluster. */
+    void stopStatistics() {
+        try {
+            ctx.performanceStatistics().stopCollectStatistics();
+        }
+        catch (IgniteCheckedException e) {
+            log.error("Failed to stop performance statistics.", e);
+        }
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /**
+         * @param ctx Kernal context.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log,
+                ctx.workersRegistry());
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < DFLT_FLUSH_SIZE && !isCancelled())
+                            wait();
+                    }
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+
+                flushBuffer();
+            }
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            flushBuffer();
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (SegmentedRingByteBuffer.ReadSegment seg : segs) {
+                    updateHeartbeat();
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                stopStatistics();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {

Review comment:
       It seems we can just use regular `cancel` implementation if we will catch `InterruptedException` and handle it accordingly.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454326941



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/AbstractPerformanceStatisticsTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.lang.management.ThreadInfo;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.PERF_STAT_DIR;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatisticsWriter.WRITER_THREAD_NAME;
+import static org.apache.ignite.internal.processors.performancestatistics.TestFilePerformanceStatisticsReader.readToLog;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/**
+ * Ignite performance statistics abstract test.
+ */
+public abstract class AbstractPerformanceStatisticsTest extends GridCommonAbstractTest {
+    /** */
+    public static final long TIMEOUT = 30_000;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false));
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+
+        U.delete(U.resolveWorkDirectory(U.defaultWorkDirectory(), PERF_STAT_DIR, false));
+    }
+
+    /** Starts collecting performance statistics. */
+    protected static void startCollectStatistics() throws Exception {
+        List<Ignite> grids = G.allGrids();
+
+        assertFalse(grids.isEmpty());
+
+        IgniteEx ignite = (IgniteEx)grids.get(0);
+
+        ignite.context().performanceStatistics().startCollectStatistics();

Review comment:
       Let's use `PerformanceStatisticsMBean#start` here and in the other 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.

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446119982



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWalker.java
##########
@@ -0,0 +1,286 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.performancestatistics.IgnitePerformanceStatistics.CacheOperationType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readIgniteUuid;
+import static org.apache.ignite.internal.processors.performancestatistics.FilePerformanceStatistics.readUuid;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatistics
+ */
+public class FilePerformanceStatisticsWalker {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -198,54 +201,74 @@ protected void removeQueryFuture(long reqId) {
         assert req.mvccSnapshot() != null || !cctx.mvccEnabled() || req.cancel() ||
             (req.type() == null && !req.fields()) : req; // Last assertion means next page request.
 
-        if (req.cancel()) {
-            cancelIds.add(new CancelMessageId(req.id(), sndId));
+        boolean performanceStatsEnabled = cctx.kernalContext().performanceStatistics().enabled();
 
-            if (req.fields())
-                removeFieldsQueryResult(sndId, req.id());
-            else
-                removeQueryResult(sndId, req.id());
-        }
-        else {
-            if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
-                if (!F.eq(req.cacheName(), cctx.name())) {
-                    GridCacheQueryResponse res = new GridCacheQueryResponse(
-                        cctx.cacheId(),
-                        req.id(),
-                        new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
-                            ", actual=" + req.cacheName()),
-                        cctx.deploymentEnabled());
+        if (performanceStatsEnabled)
+            IoStatisticsQueryHelper.startGatheringQueryStatistics();
 
-                    sendQueryResponse(sndId, res, 0);
-                }
-                else {
-                    threads.put(req.id(), Thread.currentThread());
+        try {
+            if (req.cancel()) {
+                cancelIds.add(new CancelMessageId(req.id(), sndId));
 
-                    try {
-                        GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
+                if (req.fields())
+                    removeFieldsQueryResult(sndId, req.id());
+                else
+                    removeQueryResult(sndId, req.id());
+            }
+            else {

Review comment:
       Fixed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
##########
@@ -198,54 +201,74 @@ protected void removeQueryFuture(long reqId) {
         assert req.mvccSnapshot() != null || !cctx.mvccEnabled() || req.cancel() ||
             (req.type() == null && !req.fields()) : req; // Last assertion means next page request.
 
-        if (req.cancel()) {
-            cancelIds.add(new CancelMessageId(req.id(), sndId));
+        boolean performanceStatsEnabled = cctx.kernalContext().performanceStatistics().enabled();
 
-            if (req.fields())
-                removeFieldsQueryResult(sndId, req.id());
-            else
-                removeQueryResult(sndId, req.id());
-        }
-        else {
-            if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
-                if (!F.eq(req.cacheName(), cctx.name())) {
-                    GridCacheQueryResponse res = new GridCacheQueryResponse(
-                        cctx.cacheId(),
-                        req.id(),
-                        new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
-                            ", actual=" + req.cacheName()),
-                        cctx.deploymentEnabled());
+        if (performanceStatsEnabled)
+            IoStatisticsQueryHelper.startGatheringQueryStatistics();
 
-                    sendQueryResponse(sndId, res, 0);
-                }
-                else {
-                    threads.put(req.id(), Thread.currentThread());
+        try {
+            if (req.cancel()) {
+                cancelIds.add(new CancelMessageId(req.id(), sndId));
 
-                    try {
-                        GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
+                if (req.fields())
+                    removeFieldsQueryResult(sndId, req.id());
+                else
+                    removeQueryResult(sndId, req.id());
+            }
+            else {
+                if (!cancelIds.contains(new CancelMessageId(req.id(), sndId))) {
+                    if (!F.eq(req.cacheName(), cctx.name())) {
+                        GridCacheQueryResponse res = new GridCacheQueryResponse(
+                            cctx.cacheId(),
+                            req.id(),
+                            new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
+                                ", actual=" + req.cacheName()),
+                            cctx.deploymentEnabled());
+
+                        sendQueryResponse(sndId, res, 0);
+                    }
+                    else {
+                        threads.put(req.id(), Thread.currentThread());
 
-                        if (info == null)
-                            return;
+                        try {
+                            GridCacheQueryInfo info = distributedQueryInfo(sndId, req);
 
-                        if (req.fields())
-                            runFieldsQuery(info);
-                        else
-                            runQuery(info);
-                    }
-                    catch (Throwable e) {
-                        U.error(log(), "Failed to run query.", e);
+                            if (info == null)
+                                return;
+
+                            if (req.fields())
+                                runFieldsQuery(info);
+                            else
+                                runQuery(info);
+                        }
+                        catch (Throwable e) {
+                            U.error(log(), "Failed to run query.", e);
 
-                        sendQueryResponse(sndId, new GridCacheQueryResponse(cctx.cacheId(), req.id(), e.getCause(),
-                            cctx.deploymentEnabled()), 0);
+                            sendQueryResponse(sndId, new GridCacheQueryResponse(cctx.cacheId(), req.id(), e.getCause(),
+                                cctx.deploymentEnabled()), 0);
 
-                        if (e instanceof Error)
-                            throw (Error)e;
-                    }
-                    finally {
-                        threads.remove(req.id());
+                            if (e instanceof Error)
+                                throw (Error)e;
+                        }
+                        finally {
+                            threads.remove(req.id());
+                        }
                     }
                 }
             }
+        } finally {

Review comment:
       Fixed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449436998



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,559 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;

Review comment:
       we can check writer inside doWrite




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453703597



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");
+            }
+        }
+        catch (Exception e) {
+            log.error("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops performance statistics writer. */
+    private void stopWriter() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            writer.stop();
+
+            writer = null;
+
+            log.info("Performance statistics writer stopped.");

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");
+            }
+        }
+        catch (Exception e) {
+            log.error("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops performance statistics writer. */
+    private void stopWriter() {
+        synchronized (this) {

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.

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



[GitHub] [ignite] nizhikov commented on pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#issuecomment-659308906


   Let's add tests to check tracking of `INSERT` and `UPDATE` queries.


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r446137769



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/IgnitePerformanceStatisticsMBean.java
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.processors.performancestatistics;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.mxbean.MXBeanDescription;
+
+/**
+ * MBean that provides access to performance statistics management.
+ */
+@MXBeanDescription("MBean provide access to performance statistics management.")
+public interface IgnitePerformanceStatisticsMBean {

Review comment:
       Let's renami this to `PerformanceStatisticsMBean`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453685545



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");

Review comment:
       We should log outside of the `synchronized` section.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/PerformaceStatisticsProcessor.java
##########
@@ -0,0 +1,262 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.UUID;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteFeatures;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorage;
+import org.apache.ignite.internal.processors.metastorage.DistributedMetastorageLifecycleListener;
+import org.apache.ignite.internal.processors.metastorage.ReadableDistributedMetaStorage;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.IgniteFeatures.allNodesSupports;
+
+/**
+ * Performance statistics processor.
+ * <p>
+ * Manages collecting performance statistics.
+ *
+ * @see FilePerformanceStatisticsWriter
+ * @see FilePerformanceStatisticsReader
+ */
+public class PerformaceStatisticsProcessor extends GridProcessorAdapter {
+    /** Prefix for performance statistics enabled property name. */
+    private static final String PERFORMANCE_STAT_ENABLED_PREFIX = "performanceStatistics.enabled";
+
+    /** Performance statistics writer. */
+    @Nullable private volatile FilePerformanceStatisticsWriter writer;
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Metastorage with the write access. */
+    @Nullable private volatile DistributedMetaStorage metastorage;
+
+    /** @param ctx Kernal context. */
+    public PerformaceStatisticsProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        ctx.internalSubscriptionProcessor().registerDistributedMetastorageListener(
+            new DistributedMetastorageLifecycleListener() {
+            @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
+                metastorage.listen(PERFORMANCE_STAT_ENABLED_PREFIX::equals, (key, oldVal, newVal) -> {
+                    // Skip history on local join.
+                    if (!ctx.discovery().localJoinFuture().isDone())
+                        return;
+
+                    onMetastorageUpdate((boolean)newVal);
+                });
+            }
+
+            @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+                PerformaceStatisticsProcessor.this.metastorage = metastorage;
+
+                try {
+                    Boolean performanceStatsEnabled = metastorage.read(PERFORMANCE_STAT_ENABLED_PREFIX);
+
+                    if (performanceStatsEnabled == null)
+                        return;
+
+                    onMetastorageUpdate(performanceStatsEnabled);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        });
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        write(writer -> writer.cacheOperation(type, cacheId, startTime, duration));
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        write(writer -> writer.transaction(cacheIds, startTime, duration, commited));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        write(writer -> writer.query(type, text, id, startTime, duration, success));
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        write(writer -> writer.queryReads(type, queryNodeId, id, logicalReads, physicalReads));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        write(writer -> writer.task(sesId, taskName, startTime, duration, affPartId));
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        write(writer -> writer.job(sesId, queuedTime, startTime, duration, timedOut));
+    }
+
+    /**
+     * Starts collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If starting failed.
+     */
+    public void startCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (!allNodesSupports(ctx.discovery().allNodes(), IgniteFeatures.PERFORMANCE_STATISTICS))
+            throw new IllegalStateException("Not all nodes in the cluster support collecting performance statistics.");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, true);
+    }
+
+    /**
+     * Stops collecting performance statistics.
+     *
+     * @throws IgniteCheckedException If stopping failed.
+     */
+    public void stopCollectStatistics() throws IgniteCheckedException {
+        A.notNull(metastorage, "Metastorage not ready. Node not started?");
+
+        if (ctx.isStopping())
+            throw new NodeStoppingException("Operation has been cancelled (node is stopping)");
+
+        metastorage.write(PERFORMANCE_STAT_ENABLED_PREFIX, false);
+    }
+
+    /** @return {@code True} if collecting performance statistics is enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        if (enabled())
+            stopWriter();
+    }
+
+    /** Starts or stops collecting statistics on metastorage update. */
+    private void onMetastorageUpdate(boolean start) {
+        ctx.closure().runLocalSafe(() -> {
+            if (start)
+                startWriter();
+            else
+                stopWriter();
+        });
+    }
+
+    /** Starts performance statistics writer. */
+    private void startWriter() {
+        try {
+            synchronized (this) {
+                if (enabled)
+                    return;
+
+                writer = new FilePerformanceStatisticsWriter(ctx);
+
+                writer.start();
+
+                enabled = true;
+
+                log.info("Performance statistics writer started.");
+            }
+        }
+        catch (Exception e) {
+            log.error("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops performance statistics writer. */
+    private void stopWriter() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+
+            writer.stop();
+
+            writer = null;
+
+            log.info("Performance statistics writer stopped.");

Review comment:
       We should log outside of `synchronized` section.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449567735



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {
+                    updateHeartbeat();
+
+                    SegmentedRingByteBuffer.ReadSegment seg = segs.get(i);
+
+                    try {
+                        readyForFlushSize.addAndGet(-seg.buffer().remaining());
+
+                        fileIo.writeFully(seg.buffer());
+                    }
+                    finally {
+                        seg.release();
+                    }
+                }
+
+                fileIo.force();
+            } catch (IOException e) {
+                log.error("Unable to write to file. Performance statistics collecting will be stopped.", e);
+
+                fileWriter.shutdown();
+
+                stopStatistics();
+            }
+        }
+
+        /** Shutted down the worker. */
+        private void shutdown() {

Review comment:
       We should override or use existing `cancel` method to stop this worker.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450810520



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,539 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.cancel();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(OperationType type, int cacheId, long startTime, long duration) {
+        doWrite(type,
+            () -> 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(commited ? TX_COMMIT : TX_ROLLBACK,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        boolean needWriteStr = !stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        boolean needWriteStr = !stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(sizeSupplier.getAsInt() + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)op.ordinal());
+
+        writer.accept(buf);
+
+        seg.release();
+    }
+
+    /** @return {@code True} if string is cached. {@code False} if need write string.  */
+    private boolean stringCached(String str) {
+        boolean cached = cachedStrings.contains(str.hashCode());
+
+        if (!cached)
+            cachedStrings.add(str.hashCode());
+
+        return cached;
+    }
+
+    /** @return Performance statistics file. */
+    private static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    private static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    private static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {
+        return enabled;
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            try {
+                while (!isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        synchronized (this) {
+                            while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                                wait();
+                        }
+                    }
+                    finally {
+                        blockingSectionEnd();
+                    }
+
+                    flushBuffer();
+                }
+            }
+            finally {
+                fileWriter = null;
+
+                ringByteBuffer.close();
+
+                // Make sure that all producers released their buffers to safe deallocate memory.
+                flushBuffer();
+
+                ringByteBuffer.free();
+
+                U.closeQuiet(fileIo);
+
+                cachedStrings.clear();
+
+                log.info("Performance statistics writer stopped.");
+            }
+        }
+
+        /** @return Write segment.*/
+        SegmentedRingByteBuffer.WriteSegment writeSegment(int size) {
+            SegmentedRingByteBuffer.WriteSegment seg = ringByteBuffer.offer(size);
+
+            if (seg != null) {
+                int readySize = readyForFlushSize.addAndGet(size);
+
+                if (readySize >= DFLT_FLUSH_SIZE) {
+                    synchronized (this) {
+                        // Required to start writing data to the file.
+                        notify();
+                    }
+                }
+            }
+
+            return seg;
+        }
+
+        /** Flushes to disk available bytes from the ring buffer. */
+        private void flushBuffer() {
+            List<SegmentedRingByteBuffer.ReadSegment> segs = ringByteBuffer.poll();
+
+            if (segs == null)
+                return;
+
+            try {
+                for (int i = 0; i < segs.size(); i++) {

Review comment:
       We can use foreach loop here.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449063154



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();
+                }
+                finally {
+                    blockingSectionEnd();
+                }
+            }
+
+            fileWriter = null;
+
+            ringByteBuffer.close();
+
+            // Make sure that all producers released their buffers to safe deallocate memory.
+            ringByteBuffer.poll();
+
+            ringByteBuffer.free();
+
+            U.closeQuiet(fileIo);
+
+            stringIds.clear();
+
+            stopFut.onDone();
+
+            log.info("Performance statistics writer stopped.");
+        }
+
+        /** @return Unique per file string identifier. {@code Null} if there is no cached identifier. */
+        Short stringId(String str) {
+            return stringIds.get(str);
+        }
+
+        /** @return Generate unique per file string identifier. {@code -1} if max cached limit exceeded. */
+        short generateStringId(String str) {
+            if (idsGen.get() > MAX_CACHED_STRING_COUNT)

Review comment:
       I have implemented with `Set` of hashcodes




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r443578666



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
##########
@@ -6794,6 +6899,55 @@ public InvokeAllTimeStatClosure(CacheMetricsImpl metrics, final long start) {
         }
     }
 
+    /** */
+    private class ProfileClosure<T> implements CI1<IgniteInternalFuture<T>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Operation type. */
+        private final CacheOperationType op;
+
+        /** Start time in nanoseconds. */
+        private final long start;
+
+        /**
+         * @param op Operation type.
+         * @param start Start time in nanoseconds.
+         */
+        public ProfileClosure(CacheOperationType op, long start) {
+            this.op = op;
+            this.start = start;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void apply(IgniteInternalFuture<T> fut) {
+            try {
+                if (!fut.isCancelled()) {

Review comment:
       Why we ignore cancelled operations?
   We should gather statistics about all operations that user initiate no matter if it succeeds or failed.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453723947



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics file writer worker. */
+    private final FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private final FileIO fileIo;
+
+    /** File write buffer. */
+    private final SegmentedRingByteBuffer ringByteBuffer;

Review comment:
       we use `buffer` here but `buf` in `smallBufLogged`.
   Let's use `buf` every where.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454925212



##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/performancestatistics/PerformanceStatisticsSelfTest.java
##########
@@ -0,0 +1,290 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.Lock;
+import java.util.function.Consumer;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.junits.GridAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_GET_AND_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_INVOKE_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_LOCK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_PUT_ALL;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.CACHE_REMOVE_ALL;
+
+/**
+ * Tests performance statistics.
+ */
+@SuppressWarnings({"LockAcquiredButNotSafelyReleased"})
+public class PerformanceStatisticsSelfTest extends AbstractPerformanceStatisticsTest {
+    /** Test entry processor. */
+    private static final EntryProcessor<Object, Object, Object> ENTRY_PROC =
+        new EntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Test cache entry processor. */
+    private static final CacheEntryProcessor<Object, Object, Object> CACHE_ENTRY_PROC =
+        new CacheEntryProcessor<Object, Object, Object>() {
+        @Override public Object process(MutableEntry<Object, Object> entry, Object... arguments)
+            throws EntryProcessorException {
+            return null;
+        }
+    };
+
+    /** Cache entry count. */
+    private static final int ENTRY_COUNT = 100;
+
+    /** Ignite. */
+    private static IgniteEx ignite;
+
+    /** Test cache. */
+    private static IgniteCache<Object, Object> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        log = new ListeningTestLogger(GridAbstractTest.log);
+
+        ignite = startGrid(0);
+
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        cache = ignite.cache(DEFAULT_CACHE_NAME);
+
+        for (int i = 0; i < ENTRY_COUNT; i++)
+            cache.put(i, i);
+    }
+
+    /** @throws Exception If failed. */
+    @Test
+    public void testCompute() throws Exception {
+        String testTaskName = "testTask";
+        int executions = 5;
+
+        startCollectStatistics();
+
+        IgniteRunnable task = new IgniteRunnable() {
+            @Override public void run() {
+                // No-op.
+            }
+        };
+
+        for (int i = 0; i < executions; i++)
+            ignite.compute().withName(testTaskName).run(task);
+
+        HashMap<IgniteUuid, Integer> sessions = new HashMap<>();
+        AtomicInteger tasks = new AtomicInteger();
+        AtomicInteger jobs = new AtomicInteger();
+
+        stopCollectStatisticsAndRead(new TestHandler() {
+            @Override public void task(UUID nodeId, IgniteUuid sesId, String taskName, long startTime, long duration,
+                int affPartId) {
+                sessions.compute(sesId, (uuid, val) -> val == null ? 1 : ++val);
+
+                tasks.incrementAndGet();
+
+                assertEquals(ignite.context().localNodeId(), nodeId);
+                assertEquals(testTaskName, taskName);
+                assertTrue(startTime > 0);
+                assertTrue(duration >= 0);
+                assertEquals(-1, affPartId);
+            }
+
+            @Override public void job(UUID nodeId, IgniteUuid sesId, long queuedTime, long startTime, long duration,
+            boolean timedOut) {

Review comment:
       Typo: formatting.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448865941



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/package-info.java
##########
@@ -0,0 +1,23 @@
+/*
+ * 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 description. -->
+ * Contains classes for Ignite performance statistics.
+ */
+
+package org.apache.ignite.internal.processors.performancestatistics;

Review comment:
       We don't have this file for any other internal package.
   So I think it's OK to remove it here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r450785317



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,375 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = 8 * 1024 * 1024;

Review comment:
       Let's use MB constant here.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r543192693



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
##########
@@ -761,4 +761,11 @@
      * @return Thread pool for create/rebuild indexes.
      */
     public ExecutorService buildIndexExecutorService();
+
+    /**
+     * Gets Performance statistics processor.
+     *
+     * @return Performance statistics processor.
+     */
+    public PerformanceStatisticsProcessor performanceStatistics();

Review comment:
       Let's add @IgniteExperimental for this method.




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

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



[GitHub] [ignite] nizhikov merged pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov merged pull request #7693:
URL: https://github.com/apache/ignite/pull/7693


   


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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r448990786



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            return fileWriter.shutdown();
+
+        return new GridFinishedFuture<>();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        int size = /*type*/ 1 +
+            /*cacheId*/ 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.CACHE_OPERATION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.putInt(cacheId);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+
+        seg.release();
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        int size = /*cacheIds*/ 4 + cacheIds.size() * 4 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*commit*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TRANSACTION, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.putInt(cacheIds.size());
+
+        GridIntIterator iter = cacheIds.iterator();
+
+        while (iter.hasNext())
+            buf.putInt(iter.next());
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(commited ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(text);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*type*/ 1 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*id*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*success*/ 1;
+
+        if (needWriteStr) {
+            strBytes = text.getBytes();
+
+            size += /*text*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(text);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(id);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(success ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        int size = /*type*/ 1 +
+            /*queryNodeId*/ 16 +
+            /*id*/ 8 +
+            /*logicalReads*/ 8 +
+            /*physicalReads*/ 8;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.QUERY_READS, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+        writeUuid(buf, queryNodeId);
+        buf.putLong(id);
+        buf.putLong(logicalReads);
+        buf.putLong(physicalReads);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        Short strId = writer.stringId(taskName);
+
+        boolean needWriteStr = strId == null;
+
+        byte[] strBytes = null;
+
+        int size = /*sesId*/ 24 +
+            /*compactStringFlag*/ 1 +
+            /*strId*/ 2 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*affPartId*/ 4;
+
+        if (needWriteStr) {
+            strBytes = taskName.getBytes();
+
+            size += /*taskName*/ 4 + strBytes.length;
+
+            strId = writer.generateStringId(taskName);
+        }
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.TASK, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.put(needWriteStr ? (byte)1 : 0);
+        buf.putShort(strId);
+
+        if (needWriteStr) {
+            buf.putInt(strBytes.length);
+            buf.put(strBytes);
+        }
+
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.putInt(affPartId);
+
+        seg.release();
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        int size = /*sesId*/ 24 +
+            /*queuedTime*/ 8 +
+            /*startTime*/ 8 +
+            /*duration*/ 8 +
+            /*timedOut*/ 1;
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(OperationType.JOB, size);
+
+        if (seg == null)
+            return;
+
+        ByteBuffer buf = seg.buffer();
+
+        writeIgniteUuid(buf, sesId);
+        buf.putLong(queuedTime);
+        buf.putLong(startTime);
+        buf.putLong(duration);
+        buf.put(timedOut ? (byte)1 : 0);
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(OperationType type, int size) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return null;
+
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Worker to write to performance statistics file. */
+    private class FileWriter extends GridWorker {
+        /** Maximum cached string count. */
+        private static final short MAX_CACHED_STRING_COUNT = Short.MAX_VALUE;
+
+        /** Performance statistics file I/O. */
+        private final FileIO fileIo;
+
+        /** File write buffer. */
+        private final SegmentedRingByteBuffer ringByteBuffer;
+
+        /** Minimal batch size to flush in bytes. */
+        private final int flushBatchSize;
+
+        /** Size of ready for flushing bytes. */
+        private final AtomicInteger readyForFlushSize = new AtomicInteger();
+
+        /** Stop file writer future. */
+        GridFutureAdapter<Void> stopFut = new GridFutureAdapter<>();
+
+        /** Cached strings by id. */
+        private final ConcurrentHashMap<String, Short> stringIds = new ConcurrentHashMap<>();
+
+        /** String id generator. */
+        private final AtomicInteger idsGen = new AtomicInteger();
+
+        /** {@code True} if the small buffer warning message logged. */
+        private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+        /** {@code True} if worker stopped due to maximum file size reached. */
+        private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+        /**
+         * @param ctx Kernal context.
+         * @param fileIo Performance statistics file I/O.
+         * @param maxFileSize Maximum file size in bytes.
+         * @param bufferSize Off heap buffer size in bytes.
+         * @param flushBatchSize Minimal batch size to flush in bytes.
+         * @param log Logger.
+         */
+        FileWriter(GridKernalContext ctx, FileIO fileIo, long maxFileSize, int bufferSize, int flushBatchSize,
+            IgniteLogger log) {
+            super(ctx.igniteInstanceName(), "performance-statistics-writer%" + ctx.igniteInstanceName(), log);
+
+            this.fileIo = fileIo;
+            this.flushBatchSize = flushBatchSize;
+
+            ringByteBuffer = new SegmentedRingByteBuffer(bufferSize, maxFileSize, BufferMode.DIRECT);
+
+            ringByteBuffer.init(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
+            while (!isCancelled() && !Thread.interrupted()) {
+                blockingSectionBegin();
+
+                try {
+                    synchronized (this) {
+                        while (readyForFlushSize.get() < flushBatchSize && !isCancelled())
+                            wait();
+                    }
+
+                    flushBuffer();

Review comment:
       I have moved buffer flushing out of the blocking section.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,660 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean performanceStatisticsEnabled() {
+        return enabled;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                writer.shutdown().get();
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        assert fileWriter == null;
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileIo.position(0);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public IgniteInternalFuture<Void> stop() {
+        synchronized (this) {
+            if (!enabled)
+                return new GridFinishedFuture<>();
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449398824



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,580 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** Starts collecting performance statistics. */
+    public synchronized void start() {
+        if (enabled)
+            return;
+
+        FileWriter writer = fileWriter;
+
+        // Writer is stopping.
+        if (writer != null) {
+            try {
+                U.join(writer.runner());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to wait for previous writer stopping.", e);
+            }
+        }
+
+        try {
+            File file = statisticsFile(ctx);
+
+            U.delete(file);
+
+            FileIO fileIo = fileIoFactory.create(file);
+
+            fileWriter = new FileWriter(ctx, fileIo, DFLT_FILE_MAX_SIZE, DFLT_BUFFER_SIZE, DFLT_FLUSH_SIZE, log);
+
+            new IgniteThread(fileWriter).start();
+
+            enabled = true;
+
+            log.info("Performance statistics writer started [file=" + file.getAbsolutePath() + ']');
+        }
+        catch (IOException | IgniteCheckedException e) {
+            log.error("Failed to start performance statistics writer.", e);
+
+            throw new IgniteException("Failed to start performance statistics writer.", e);
+        }
+    }
+
+    /** Stops collecting performance statistics. */
+    public void stop() {
+        synchronized (this) {
+            if (!enabled)
+                return;
+
+            enabled = false;
+        }
+
+        log.info("Stopping performance statistics writer.");
+
+        FileWriter fileWriter = this.fileWriter;
+
+        if (fileWriter != null)
+            fileWriter.shutdown();
+    }
+
+    /**
+     * @param type Operation type.
+     * @param cacheId Cache id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     */
+    public void cacheOperation(CacheOperation type, int cacheId, long startTime, long duration) {
+        doWrite(OperationType.CACHE_OPERATION,
+            () -> 1 + 4 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                buf.putInt(cacheId);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+            });
+    }
+
+    /**
+     * @param cacheIds Cache IDs.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param commited {@code True} if commited.
+     */
+    public void transaction(GridIntList cacheIds, long startTime, long duration, boolean commited) {
+        doWrite(OperationType.TRANSACTION,
+            () -> 4 + cacheIds.size() * 4 + 8 + 8 + 1,
+            buf -> {
+                buf.putInt(cacheIds.size());
+
+                GridIntIterator iter = cacheIds.iterator();
+
+                while (iter.hasNext())
+                    buf.putInt(iter.next());
+
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(commited ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param text Query text in case of SQL query. Cache name in case of SCAN query.
+     * @param id Query id.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration in nanoseconds.
+     * @param success Success flag.
+     */
+    public void query(GridCacheQueryType type, String text, long id, long startTime, long duration, boolean success) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(text);
+        byte[] strBytes = needWriteStr ? text.getBytes() : null;
+
+        doWrite(OperationType.QUERY, () -> {
+            int size = 1 + 1 + 4 + 8 + 8 + 8 + 1;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            buf.put((byte)type.ordinal());
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(text.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(id);
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.put(success ? (byte)1 : 0);
+        });
+    }
+
+    /**
+     * @param type Cache query type.
+     * @param queryNodeId Originating node id.
+     * @param id Query id.
+     * @param logicalReads Number of logical reads.
+     * @param physicalReads Number of physical reads.
+     */
+    public void queryReads(GridCacheQueryType type, UUID queryNodeId, long id, long logicalReads, long physicalReads) {
+        doWrite(OperationType.QUERY_READS,
+            () -> 1 + 16 + 8 + 8 + 8,
+            buf -> {
+                buf.put((byte)type.ordinal());
+                writeUuid(buf, queryNodeId);
+                buf.putLong(id);
+                buf.putLong(logicalReads);
+                buf.putLong(physicalReads);
+            });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param taskName Task name.
+     * @param startTime Start time in milliseconds.
+     * @param duration Duration.
+     * @param affPartId Affinity partition id.
+     */
+    public void task(IgniteUuid sesId, String taskName, long startTime, long duration, int affPartId) {
+        FileWriter writer = fileWriter;
+
+        if (writer == null)
+            return;
+
+        boolean needWriteStr = !writer.stringCached(taskName);
+        byte[] strBytes = needWriteStr ? taskName.getBytes() : null;
+
+        doWrite(OperationType.TASK, () -> {
+            int size = 24 + 1 + 4 + 8 + 8 + 4;
+
+            if (needWriteStr)
+                size += 4 + strBytes.length;
+
+            return size;
+        }, buf -> {
+            writeIgniteUuid(buf, sesId);
+            buf.put(needWriteStr ? (byte)1 : 0);
+            buf.putInt(taskName.hashCode());
+
+            if (needWriteStr) {
+                buf.putInt(strBytes.length);
+                buf.put(strBytes);
+            }
+
+            buf.putLong(startTime);
+            buf.putLong(duration);
+            buf.putInt(affPartId);
+        });
+    }
+
+    /**
+     * @param sesId Session id.
+     * @param queuedTime Time job spent on waiting queue.
+     * @param startTime Start time in milliseconds.
+     * @param duration Job execution time.
+     * @param timedOut {@code True} if job is timed out.
+     */
+    public void job(IgniteUuid sesId, long queuedTime, long startTime, long duration, boolean timedOut) {
+        doWrite(OperationType.JOB,
+            () -> 24 + 8 + 8 + 8 + 1,
+            buf -> {
+                writeIgniteUuid(buf, sesId);
+                buf.putLong(queuedTime);
+                buf.putLong(startTime);
+                buf.putLong(duration);
+                buf.put(timedOut ? (byte)1 : 0);
+            });
+    }
+
+    /**
+     * @param op Operation type.
+     * @param sizeSupplier Record size supplier.
+     * @param writer Record writer.
+     */
+    private void doWrite(OperationType op, IntSupplier sizeSupplier, Consumer<ByteBuffer> writer) {
+        FileWriter fileWriter = this.fileWriter;
+
+        // Writer stopping.
+        if (fileWriter == null)
+            return;
+
+        int size = sizeSupplier.getAsInt();
+
+        SegmentedRingByteBuffer.WriteSegment seg = reserveBuffer(fileWriter, op, size);
+
+        if (seg == null)
+            return;
+
+        writer.accept(seg.buffer());
+
+        seg.release();
+    }
+
+    /**
+     * Reserves buffer's write segment.
+     *
+     * @param fileWriter File writer.
+     * @param type Operation type.
+     * @param size Record size.
+     * @return Buffer's write segment or {@code null} if not enought space or writer stopping.
+     */
+    private SegmentedRingByteBuffer.WriteSegment reserveBuffer(FileWriter fileWriter, OperationType type, int size) {
+        SegmentedRingByteBuffer.WriteSegment seg = fileWriter.writeSegment(size + /*type*/ 1);
+
+        if (seg == null) {
+            fileWriter.logSmallBufferMessage();
+
+            return null;
+        }
+
+        // Ring buffer closed (writer stopping) or maximum size reached.
+        if (seg.buffer() == null) {
+            seg.release();
+
+            if (!fileWriter.isCancelled())
+                fileWriter.onMaxFileSizeReached();
+
+            return null;
+        }
+
+        ByteBuffer buf = seg.buffer();
+
+        buf.put((byte)type.ordinal());
+
+        return seg;
+    }
+
+    /** @return Performance statistics file. */
+    public static File statisticsFile(GridKernalContext ctx) throws IgniteCheckedException {
+        String igniteWorkDir = U.workDirectory(ctx.config().getWorkDirectory(), ctx.config().getIgniteHome());
+
+        File fileDir = U.resolveWorkDirectory(igniteWorkDir, PERFORMANCE_STAT_DIR, false);
+
+        return new File(fileDir, "node-" + ctx.localNodeId() + ".prf");
+    }
+
+    /** Writes {@link UUID} to buffer. */
+    public static void writeUuid(ByteBuffer buf, UUID uuid) {
+        buf.putLong(uuid.getMostSignificantBits());
+        buf.putLong(uuid.getLeastSignificantBits());
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Writes {@link IgniteUuid} to buffer. */
+    public static void writeIgniteUuid(ByteBuffer buf, IgniteUuid uuid) {
+        buf.putLong(uuid.globalId().getMostSignificantBits());
+        buf.putLong(uuid.globalId().getLeastSignificantBits());
+        buf.putLong(uuid.localId());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {

Review comment:
       This method should be moved to the reader.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r454182073



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsReader.java
##########
@@ -0,0 +1,342 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.FileVisitOption;
+import java.nio.file.FileVisitResult;
+import java.nio.file.Path;
+import java.nio.file.SimpleFileVisitor;
+import java.nio.file.attribute.BasicFileAttributes;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.UUID;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.nativeOrder;
+import static java.nio.file.Files.walkFileTree;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.cacheOperation;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.transactionOperation;
+
+/**
+ * Walker over the performance statistics file.
+ *
+ * @see FilePerformanceStatisticsWriter
+ */
+public class FilePerformanceStatisticsReader {
+    /** File read buffer size. */
+    private static final int READ_BUFFER_SIZE = (int)(8 * U.MB);
+
+    /** Uuid as string pattern. */
+    private static final String UUID_STR_PATTERN =
+        "[a-fA-F0-9]{8}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{4}-[a-fA-F0-9]{12}";
+
+    /** File name pattern. */
+    private static final Pattern FILE_PATTERN = Pattern.compile("^node-(" + UUID_STR_PATTERN + ").prf$");
+
+    /** IO factory. */
+    private static final RandomAccessFileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+    /**
+     * Walks over performance statistics files.
+     *
+     * @param filesOrDirs Files or directories.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    public static void read(List<File> filesOrDirs, PerformanceStatisticsHandler... handlers) throws IOException {
+        List<File> files = resolveFiles(filesOrDirs);
+
+        if (files.isEmpty())
+            return;
+
+        for (File file : files)
+            readFile(file, handlers);
+    }
+
+    /**
+     * Walks over performance statistics file.
+     *
+     * @param file Performance statistics file.
+     * @param handlers Handlers to process deserialized operation.
+     */
+    private static void readFile(File file, PerformanceStatisticsHandler... handlers) throws IOException {
+        UUID nodeId = checkFileName(file);
+
+        ByteBuffer buf = allocateDirect(READ_BUFFER_SIZE).order(nativeOrder());
+
+        PerformanceStatisticsDeserializer des = new PerformanceStatisticsDeserializer(nodeId, handlers);
+
+        try (FileIO io = ioFactory.create(file)) {
+            while (true) {
+                int read = io.read(buf);
+
+                buf.flip();
+
+                if (read <= 0)
+                    break;
+
+                while (des.read(buf));
+
+                buf.compact();
+            }
+        }
+        finally {
+            GridUnsafe.cleanDirectBuffer(buf);
+        }
+    }
+
+    /** Resolves performance statistics files. */
+    private static List<File> resolveFiles(List<File> filesOrDirs) throws IOException {
+        if (filesOrDirs == null || filesOrDirs.isEmpty())
+            return Collections.emptyList();
+
+        List<File> files = new LinkedList<>();
+
+        for (File file : filesOrDirs) {
+            if (file.isDirectory()) {
+                walkFileTree(file.toPath(), EnumSet.noneOf(FileVisitOption.class), 1,
+                    new SimpleFileVisitor<Path>() {
+                        @Override public FileVisitResult visitFile(Path path, BasicFileAttributes attrs) {
+                            if (checkFileName(path.toFile()) != null)
+                                files.add(path.toFile());
+
+                            return FileVisitResult.CONTINUE;
+                        }
+                    });
+
+                continue;
+            }
+
+            if (checkFileName(file) != null)
+                files.add(file);
+        }
+
+        return files;
+    }
+
+    /** @return UUID node of file. {@code Null} if this is not a statistics file. */
+    @Nullable private static UUID checkFileName(File file) {
+        Matcher matcher = FILE_PATTERN.matcher(file.getName());
+
+        if (matcher.matches())
+            return UUID.fromString(matcher.group(1));
+
+        return null;
+    }
+
+    /** Reads {@link UUID} from buffer. */
+    public static UUID readUuid(ByteBuffer buf) {
+        return new UUID(buf.getLong(), buf.getLong());
+    }
+
+    /** Reads {@link IgniteUuid} from buffer. */
+    public static IgniteUuid readIgniteUuid(ByteBuffer buf) {
+        UUID globalId = new UUID(buf.getLong(), buf.getLong());
+
+        return new IgniteUuid(globalId, buf.getLong());
+    }
+
+    /** Performance statistics operations deserializer. */
+    private static class PerformanceStatisticsDeserializer {
+        /** Handlers to process deserialized operation. */
+        private final PerformanceStatisticsHandler[] handlers;
+
+        /** Node id. */
+        private final UUID nodeId;
+
+        /** @param handlers Handlers to process deserialized operation. */
+        PerformanceStatisticsDeserializer(UUID nodeId, PerformanceStatisticsHandler... handlers) {
+            this.nodeId = nodeId;
+            this.handlers = handlers;
+        }
+
+        /**
+         * Tries to deserialize performance statistics operation from buffer and notify handlers.
+         *
+         * @param buf Buffer.
+         * @return {@code True} if operation deserialized and handlers notified. {@code False} if not enough bytes.
+         */
+        boolean read(ByteBuffer buf) {
+            int pos = buf.position();
+
+            if (deserialize(buf))
+                return true;
+
+            buf.position(pos);
+
+            return false;
+        }
+
+        /**
+         * @param buf Buffer.
+         * @return {@code True} if operation deserialized. {@code False} if not enough bytes.
+         */
+        private boolean deserialize(ByteBuffer buf) {
+            if (buf.remaining() < 1)
+                return false;
+
+            byte opTypeByte = buf.get();
+
+            OperationType opType = OperationType.fromOrdinal(opTypeByte);
+
+            if (cacheOperation(opType)) {
+                if (buf.remaining() < 4 + 8 + 8)

Review comment:
       We have code to calculate record size both in the writer and in the reader.
   Let's have it in one place - `OperationType`.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453724777



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * U.GB;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = (int)(32 * U.MB);
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = (int)(8 * U.MB);
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics file writer worker. */
+    private final FileWriter fileWriter;
+
+    /** Performance statistics file I/O. */
+    private final FileIO fileIo;
+
+    /** File write buffer. */
+    private final SegmentedRingByteBuffer ringByteBuffer;
+
+    /** Count of written to buffer bytes. */
+    private final AtomicInteger writtenToBuffer = new AtomicInteger();
+
+    /** {@code True} if the small buffer warning message logged. */
+    private final AtomicBoolean smallBufLogged = new AtomicBoolean();
+
+    /** {@code True} if worker stopped due to maximum file size reached. */
+    private final AtomicBoolean stopByMaxSize = new AtomicBoolean();
+
+    /** Hashcodes of cached strings. */
+    private final ConcurrentSkipListSet<Integer> cachedStrings = new ConcurrentSkipListSet<>();
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) throws IgniteCheckedException, IOException {
+        log = ctx.log(getClass());
+
+        File file = statisticsFile(ctx);
+
+        U.delete(file);
+
+        fileIo = fileIoFactory.create(file);
+
+        log.info("Performance statistics file created [file=" + file.getAbsolutePath() + ']');
+
+        ringByteBuffer = new SegmentedRingByteBuffer(DFLT_BUFFER_SIZE, DFLT_FILE_MAX_SIZE,
+            SegmentedRingByteBuffer.BufferMode.DIRECT);
+
+        fileWriter = new FileWriter(ctx, log);
+    }
+
+    /** Starts collecting performance statistics. */
+    public void start() {

Review comment:
       Why do we need an explicit `start` method? We can start `fileWriter` thread from constructor.




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

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



[GitHub] [ignite] NSAmelchev commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r453701024



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedByInterruptException;
+import java.util.Collections;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.IntSupplier;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.JOB;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.QUERY_READS;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TASK;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_COMMIT;
+import static org.apache.ignite.internal.processors.performancestatistics.OperationType.TX_ROLLBACK;
+
+/**
+ * Performance statistics writer based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";

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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #7693: IGNITE-12666 Provide cluster performance profiling tool

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #7693:
URL: https://github.com/apache/ignite/pull/7693#discussion_r449210050



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/performancestatistics/FilePerformanceStatisticsWriter.java
##########
@@ -0,0 +1,643 @@
+/*
+ * 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.processors.performancestatistics;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer;
+import org.apache.ignite.internal.processors.cache.persistence.wal.SegmentedRingByteBuffer.BufferMode;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+import org.apache.ignite.internal.util.GridIntIterator;
+import org.apache.ignite.internal.util.GridIntList;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Performance statistics collector based on logging to a file.
+ * <p>
+ * Each node collects statistics to a file placed under {@link #PERFORMANCE_STAT_DIR}.
+ * <p>
+ * <b>Note:</b> Start again will erase previous performance statistics files.
+ * <p>
+ * To iterate over records use {@link FilePerformanceStatisticsReader}.
+ */
+public class FilePerformanceStatisticsWriter {
+    /** Default maximum file size in bytes. Performance statistics will be stopped when the size exceeded. */
+    public static final long DFLT_FILE_MAX_SIZE = 32 * 1024 * 1024 * 1024L;
+
+    /** Default off heap buffer size in bytes. */
+    public static final int DFLT_BUFFER_SIZE = 32 * 1024 * 1024;
+
+    /** Default minimal batch size to flush in bytes. */
+    public static final int DFLT_FLUSH_SIZE = 8 * 1024 * 1024;
+
+    /** Directory to store performance statistics files. Placed under Ignite work directory. */
+    public static final String PERFORMANCE_STAT_DIR = "performanceStatistics";
+
+    /** Factory to provide I/O interface. */
+    private final FileIOFactory fileIoFactory = new RandomAccessFileIOFactory();
+
+    /** Performance statistics enabled flag. */
+    private volatile boolean enabled;
+
+    /** Performance statistics file writer worker. */
+    @Nullable private volatile FileWriter fileWriter;
+
+    /** Kernal context. */
+    private final GridKernalContext ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** @param ctx Kernal context. */
+    public FilePerformanceStatisticsWriter(GridKernalContext ctx) {
+        log = ctx.log(getClass());
+
+        this.ctx = ctx;
+    }
+
+    /** @return {@code True} if collecting performance statistics enabled. */
+    public boolean enabled() {

Review comment:
       Can you, please, move this method to the bottom of the class, but above the FileWriter declaration to improve code readability.




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

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