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

[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #972: IGNITE-16657 [Native Persistence 3.0] Implement partitions chunks merger

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


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/compaction/Compactor.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.pagememory.persistence.compaction;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.stream.Collectors.toCollection;
+
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.List;
+import java.util.Objects;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.configuration.ConfigurationValue;
+import org.apache.ignite.internal.logger.IgniteLogger;
+import org.apache.ignite.internal.pagememory.io.PageIo;
+import org.apache.ignite.internal.pagememory.persistence.store.DeltaFilePageStoreIo;
+import org.apache.ignite.internal.pagememory.persistence.store.FilePageStore;
+import org.apache.ignite.internal.pagememory.persistence.store.FilePageStoreManager;
+import org.apache.ignite.internal.thread.IgniteThread;
+import org.apache.ignite.internal.thread.NamedThreadFactory;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.worker.IgniteWorker;
+import org.apache.ignite.internal.util.worker.IgniteWorkerListener;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Entity to compact delta files.
+ *
+ * <p>To start compacting delta files, you need to notify about the appearance of {@link #addDeltaFiles(int) delta files ready for
+ * compaction}. Then all delta files {@link FilePageStore#getDeltaFileToCompaction() ready for compaction} will be collected and merged with
+ * their {@link FilePageStore file page stores} until all delta files are compacted.
+ *
+ * <p>Delta file compaction process consists of:
+ * <ul>
+ *  <li>Copying pages from a delta file to a partition file.</li>
+ *  <li>Fsync of the partition file.</li>
+ *  <li>Remove delta file from {@link FilePageStore} and file system.</li>
+ * </ul>
+ */
+public class Compactor extends IgniteWorker {
+    private final Object mux = new Object();
+
+    private final @Nullable ThreadPoolExecutor threadPoolExecutor;
+
+    private final AtomicInteger deltaFileCount = new AtomicInteger();
+
+    private final FilePageStoreManager filePageStoreManager;
+
+    /** Thread local with buffers for the compaction threads. */
+    private final ThreadLocal<ByteBuffer> threadBuf;
+
+    /**
+     * Creates new ignite worker with given parameters.
+     *
+     * @param log Logger.
+     * @param igniteInstanceName Name of the Ignite instance this runnable is used in.
+     * @param listener Listener for life-cycle events.
+     * @param threads Number of compaction threads.
+     * @param filePageStoreManager File page store manager.
+     * @param pageSize Page size in bytes.
+     */
+    public Compactor(
+            IgniteLogger log,
+            String igniteInstanceName,
+            @Nullable IgniteWorkerListener listener,
+            ConfigurationValue<Integer> threads,
+            FilePageStoreManager filePageStoreManager,
+            int pageSize
+    ) {
+        super(log, igniteInstanceName, "compaction-thread", listener);
+
+        this.filePageStoreManager = filePageStoreManager;
+
+        threadBuf = ThreadLocal.withInitial(() -> {
+            ByteBuffer tmpWriteBuf = ByteBuffer.allocateDirect(pageSize);
+
+            tmpWriteBuf.order(ByteOrder.nativeOrder());
+
+            return tmpWriteBuf;
+        });
+
+        int threadCount = threads.value();
+
+        if (threadCount > 1) {
+            threadPoolExecutor = new ThreadPoolExecutor(
+                    threadCount,
+                    threadCount,
+                    30_000,
+                    MILLISECONDS,
+                    new LinkedBlockingQueue<>(),
+                    new NamedThreadFactory("compaction-runner-io", log)
+            );
+        } else {
+            threadPoolExecutor = null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    protected void body() throws InterruptedException {
+        try {
+            while (!isCancelled()) {
+                waitDeltaFiles();
+
+                if (isCancelled()) {
+                    log.info("Skipping the delta file compaction because the node is stopping");
+
+                    return;
+                }
+
+                doCompaction();
+            }
+        } catch (Throwable t) {
+            // TODO: IGNITE-16899 By analogy with 2.0, we need to handle the exception (err) by the FailureProcessor
+
+            throw new IgniteInternalException(t);
+        }
+    }
+
+    /**
+     * Waiting for delta files.
+     */
+    void waitDeltaFiles() {
+        try {
+            synchronized (mux) {
+                while (deltaFileCount.get() == 0 && !isCancelled()) {
+                    blockingSectionBegin();
+
+                    try {
+                        mux.wait();
+                    } finally {
+                        blockingSectionEnd();
+                    }
+                }
+            }
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            isCancelled.set(true);
+        }
+    }
+
+    /**
+     * Adds the number of delta files to compact.
+     *
+     * @param count Number of delta files.
+     */
+    public void addDeltaFiles(int count) {
+        assert count >= 0;
+
+        if (count > 0) {
+            deltaFileCount.addAndGet(count);
+
+            synchronized (mux) {
+                mux.notifyAll();
+            }
+        }
+    }
+
+    /**
+     * Merges delta files with partition files.
+     */
+    void doCompaction() {
+        // Let's collect one delta file for each partition.
+        Queue<IgniteBiTuple<FilePageStore, DeltaFilePageStoreIo>> queue = filePageStoreManager.allPageStores().stream()
+                .flatMap(List::stream)
+                .map(filePageStore -> {
+                    DeltaFilePageStoreIo deltaFileToCompaction = filePageStore.getDeltaFileToCompaction();
+
+                    return deltaFileToCompaction == null ? null : new IgniteBiTuple<>(filePageStore, deltaFileToCompaction);
+                })
+                .filter(Objects::nonNull)
+                .collect(toCollection(ConcurrentLinkedQueue::new));
+
+        assert !queue.isEmpty();
+
+        updateHeartbeat();
+
+        int threads = threadPoolExecutor == null ? 1 : threadPoolExecutor.getMaximumPoolSize();
+
+        CompletableFuture<?>[] futures = new CompletableFuture[threads];
+
+        for (int i = 0; i < threads; i++) {
+            CompletableFuture<?> future = futures[i] = new CompletableFuture<>();

Review Comment:
   nice



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

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

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