You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/09/20 05:59:20 UTC

[GitHub] [ignite] Berkof opened a new pull request #9423: IGNITE-15281 New implementation of local statistics collection.

Berkof opened a new pull request #9423:
URL: https://github.com/apache/ignite/pull/9423


   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;

Review comment:
       I see that IgniteStatisticsManagerImpl create TWO thread pools which uses to execute task with critical sections in the class. Should they work concurrently?




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

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

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
##########
@@ -234,6 +234,37 @@ public byte maxPartitionObsolescencePercent() {
         return S.toString(StatisticsObjectConfiguration.class, this);
     }
 
+    /**
+     * Compare only configuration from the same branch. I.e. can't correctly compare
+     * Cfg(A=v1,B=v3) vs Cfg(A=v2,B=v1)
+     * Cfg(A=v1,B=v3) vs Cfg(A=v1m C=v2)
+     * because there is no changes chain to get one from another.
+     *
+     * @param o Other configuration to compare.
+     * @return Comparison result.
+     */
+    @Override public int compareTo(@NotNull StatisticsObjectConfiguration o) {
+        if (this == o)
+            return 0;
+
+        if (cols.size() < o.cols.size())

Review comment:
       many places in this class cannot handle the `null` column map. This one is one of those places

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsStorageAbstractTest.java
##########
@@ -45,11 +45,13 @@
         super.beforeTestsStarted();
         cleanPersistenceDir();
 
-        //startGridsMultiThreaded(1);
         startGrid(0);
         startGrid(1);
+
         grid(0).cluster().state(ClusterState.ACTIVE);
 
+        Thread.sleep(500);

Review comment:
       why do we need this `sleep`?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {
-                        try {
-                            onChangeStatisticConfiguration(
-                                (StatisticsObjectConfiguration)oldV,
-                                (StatisticsObjectConfiguration)newV
-                            );
-                        }
-                        catch (Throwable e) {
-                            log.warning("Unexpected exception on change statistic configuration [old="
-                                + oldV + ", new=" + newV + ']', e);
-                        }
+                        StatisticsObjectConfiguration newStatCfg = (StatisticsObjectConfiguration)newV;
+
+                        statProc.busyRun(() -> updateLocalStatistics(newStatCfg));
+
                     });
                 }
             );
         }
     };
 
-    /** Exchange listener. */
-    private final PartitionsExchangeAware exchAwareLsnr = new PartitionsExchangeAware() {
-        @Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
-            started = true;
-
-            // Skip join/left client nodes.
-            if (fut.exchangeType() != GridDhtPartitionsExchangeFuture.ExchangeType.ALL ||
-                cluster.clusterState().lastState() != ClusterState.ACTIVE)
-                return;
+    /**
+     * Update statistics after topology change, if necessary.
+     *
+     * @param fut Topology change future.
+     */
+    public void afterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {

Review comment:
       please move all methods below constructor

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {

Review comment:
       unused

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
##########
@@ -169,6 +169,9 @@ public boolean collectionAwareEqual(StatisticsColumnConfiguration o) {
      */
     public StatisticsColumnConfiguration createTombstone()
     {

Review comment:
       C-style braces

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
##########
@@ -23,7 +23,7 @@
 import org.apache.ignite.internal.util.collection.IntMap;
 
 /**
- * Statistics store interface.
+ * Statistics store interface. Wrap phisical storage (in memory or local metastore) to give easy access to statistics.

Review comment:
       physical

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();

Review comment:
       why it's static?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {
+        return gatheringInProgress.get(key);
+    }
+
+    /**
+     * Start gathering.
+     */
+    public void start() {
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering started.");
+
+        active = true;
+    }
+
+    /**
+     * Stop gathering.
+     */
+    public void stop() {
+        if (log.isTraceEnabled())
+            log.trace(String.format("Statistics gathering stopping %d task...", gatheringInProgress.size()));
+
+        active = false;
+
+        cancelAllTasks();
+
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering stopped.");
+    }
+
+    /**
+     * Cancel all currently running statistics gathering tasks.
+     */
+    public void cancelAllTasks() {
+        gatheringInProgress.values().forEach(g -> g.future().cancel(true));
+        // Can skip waiting for each task finished because of global busyLock.
+
+        gatheringInProgress.clear();
+
+        busyLock.block();
+        busyLock.unblock();
+    }
+
+    /**
+     * Get gathering pool.
+     *
+     * @return Gathering pool.
+     */
+    public IgniteThreadPoolExecutor gatheringPool() {

Review comment:
       unused

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();

Review comment:
       `mux` is not used anymore

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {

Review comment:
       execute




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -528,47 +444,69 @@ public void updateObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Intege
 
         for (Map.Entry<StatisticsKey, Set<Integer>> objDeleted : deleted.entrySet())
             store.clearObsolescenceInfo(objDeleted.getKey(), objDeleted.getValue());
+
+        fitObsolescenceInfo(cfg);
     }
 
     /**
-     * Load or update obsolescence info cache to fit specified cfg.
+     * Check store to clean unnecessary records.
      *
      * @param cfg Map object statistics configuration to primary partitions set.
      */
-    public synchronized void checkObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {
-        if (!started.compareAndSet(false, true))
-            loadObsolescenceInfo(cfg);
-        else
-            updateObsolescenceInfo(cfg);
+    private void fitObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {

Review comment:
       Why don't use BitSet instead of collections of boxed Integers.
   It is ok to do in separate task.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
##########
@@ -203,11 +239,43 @@ public IgniteStatisticsInMemoryStoreImpl(Function<Class<?>, IgniteLogger> logSup
         Collection<ObjectPartitionStatisticsImpl> statistics
     ) {
         IntMap<ObjectPartitionStatisticsImpl> statisticsMap = new IntHashMap<ObjectPartitionStatisticsImpl>();
+
         for (ObjectPartitionStatisticsImpl s : statistics) {
             if (statisticsMap.put(s.partId(), s) != null)
                 log.warning(String.format("Trying to save more than one %s.%s partition statistics for partition %d",
                     key.schema(), key.obj(), s.partId()));
         }
+
         return statisticsMap;
     }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadObsolescenceMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();

Review comment:
       Redundant collection.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -297,78 +332,104 @@ public IgniteStatisticsConfigurationManager statisticConfiguration() {
 
     /** {@inheritDoc} */
     @Override public StatisticsUsageState usageState() {
-        return usageState.getOrDefault(DEFAULT_STATISTICS_USAGE_STATE);
+        return (lastUsageState == null) ? DEFAULT_STATISTICS_USAGE_STATE : lastUsageState;
     }
 
     /** {@inheritDoc} */
     @Override public void onRowUpdated(String schemaName, String objName, int partId, byte[] keyBytes) {
-        try {
-            if (statCfgMgr.config(new StatisticsKey(schemaName, objName)) != null)
-                statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isInfoEnabled())
-                log.info(String.format("Error while obsolescence key in %s.%s due to %s", schemaName, objName,
-                    e.getMessage()));
-        }
+        statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
     }
 
     /**
      * Save dirty obsolescence info to local metastore. Check if statistics need to be refreshed and schedule it.
+     *
+     * 1) Get all dirty partition statistics.
+     * 2) Make separate tasks for each key to avoid saving obsolescence info for removed partition (race).
+     * 3) Check if partition should be recollected and add it to list in its tables task.
+     * 4) Submit tasks. Actually obsolescence info will be stored during task processing.
      */
     public synchronized void processObsolescence() {
-        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.saveObsolescenceInfo();
+        StatisticsUsageState state = usageState();
 
-        Map<StatisticsKey, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
-
-        if (!F.isEmpty(tasks))
-            if (log.isTraceEnabled())
-                log.trace(String.format("Refreshing statistics for %d targets", tasks.size()));
+        if (state != ON || ctx.isStopping()) {
+            if (log.isDebugEnabled())
+                log.debug("Skipping obsolescence processing.");
 
-        for (Map.Entry<StatisticsKey, List<Integer>> objTask : tasks.entrySet()) {
-            GridH2Table tbl = schemaMgr.dataTable(objTask.getKey().schema(), objTask.getKey().obj());
+            return;
+        }
 
-            if (tbl == null) {
-                if (log.isDebugEnabled())
-                    log.debug(String.format("Got obsolescence statistics for unknown table %s", objTask.getKey()));
+        if (log.isTraceEnabled())
+            log.trace("Process statistics obsolescence started.");
 
-                continue;
-            }
+        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.getDirtyObsolescenceInfo();
 
-            StatisticsObjectConfiguration objCfg;
-            try {
-                objCfg = statCfgMgr.config(objTask.getKey());
-            } catch (IgniteCheckedException e) {
-                log.warning("Unable to load statistics object configuration from global metastore", e);
-                continue;
-            }
+        if (F.isEmpty(dirty)) {
+            if (log.isTraceEnabled())
+                log.trace("No dirty obsolescence info found. Finish obsolescence processing.");
 
-            if (objCfg == null) {
-                if (log.isDebugEnabled())
-                    log.debug(String.format("Got obsolescence statistics for unknown configuration %s", objTask.getKey()));
+            return;
+        }
+        else {
+            if (log.isTraceEnabled())
+                log.trace(String.format("Scheduling obsolescence savings for %d targets", dirty.size()));
+        }
 
-                continue;
-            }
+        Map<StatisticsObjectConfiguration, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
 
-            GridCacheContext cctx = tbl.cacheContext();
+        for (Map.Entry<StatisticsObjectConfiguration, List<Integer>> objTask : tasks.entrySet()) {
+            StatisticsKey key = objTask.getKey().key();
+            GridH2Table tbl = schemaMgr.dataTable(key.schema(), key.obj());
 
-            Set<Integer> parts = cctx.affinity().primaryPartitions(
-                cctx.localNodeId(), cctx.affinity().affinityTopologyVersion());
+            if (tbl == null) {
+                // Table can be removed earlier, but not already processed. Or somethink goes wrong. Try to reschedule.
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Got obsolescence statistics for unknown table %s", objTask.getKey()));
+            }
 
-            statCfgMgr.gatherLocalStatistics(objCfg, tbl, parts, new HashSet<>(objTask.getValue()), null);
+            statProc.updateKeyAsync(true, tbl, objTask.getKey(), new HashSet<>(objTask.getValue()),
+                null);
+//            if (objTask.getValue().isEmpty()) {
+//                // Just save or totally remove obsolescence info, no additional operations needed.
+//                statProc.updateKeyAsync(true, tbl, objTask.getKey(), Collections.emptySet(), null);
+//            }
+//            else {
+//                // Schedule full gathering.
+//                GridCacheContext<?, ?> cctx = (tbl == null) ? null : tbl.cacheContext();
+//
+//                AffinityTopologyVersion topVer = null;
+//
+//                if (!cctx.gate().enterIfNotStopped())
+//                    continue;
+//
+//                try {
+//                    topVer = cctx.affinity().affinityTopologyVersion();
+//                    cctx.affinity().affinityReadyFuture(topVer).get();
+//                }
+//                catch (IgniteCheckedException e) {
+//                    log.warning("Unable to get topology ready.", e);
+//                }
+//                finally {
+//                    cctx.gate().leave();
+//                }
+//
+//                statProc.updateKeyAsync(true, tbl, objTask.getKey(), new HashSet<>(objTask.getValue()),
+//                    topVer);
+//
+//            }

Review comment:
       Commented code




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -375,6 +292,15 @@ public ObjectStatisticsImpl getLocalStatistics(StatisticsKey key) {
         return locStats.get(key);
     }
 
+    /**
+     * Get all local statistics. Return internal map without copying.
+     *
+     * @return Local (for current node) object statistics.
+     */
+    public Map<StatisticsKey, ObjectStatisticsImpl> getAllLocalStatisticsInt() {

Review comment:
       ```suggestion
       public Map<StatisticsKey, ObjectStatisticsImpl> localStatisticsMap() {
   ```




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -365,6 +265,23 @@ public void saveLocalStatistics(StatisticsKey key, ObjectStatisticsImpl statisti
         locStats.put(key, statistics);
     }
 
+    /**
+     * Clear specified partition ids statistics.
+     *
+     * @param key Key to remove statistics by.
+     * @param partsToRemove Set of parititon ids to remove.
+     */
+    public void clearLocalPartitionIdsStatistics(StatisticsKey key, Set<Integer> partsToRemove) {

Review comment:
       ```suggestion
       public void clearLocalStatistics(StatisticsKey key, Set<Integer> partsToRemove) {
   ```




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -400,10 +456,21 @@ public synchronized void processObsolescence() {
                     taskParts.add(k);
             });
 
-            if (!taskParts.isEmpty())
-                res.put(key, taskParts);
+            // Will add even empty list of partitions to recollect just to force obsolescence info to be stored.
+            res.put(finalCfg, taskParts);
         }
 
         return res;
     }
+
+    /**
+     * Check that cluster statistics usage state is not OFF and cluster is active.
+     *
+     * @param op Operation name.
+     */
+    public void checkStatisticsState(String op) {

Review comment:
       ```suggestion
       public void ensureActive(String op) {
   ```




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -353,6 +252,7 @@ public ObjectPartitionStatisticsImpl getLocalPartitionStatistics(StatisticsKey k
      */
     public void clearLocalPartitionStatistics(StatisticsKey key, int partId) {

Review comment:
       ```suggestion
       public void clearLocalStatistics(StatisticsKey key, int partId) {
   ```




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is it?




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsStorageAbstractTest.java
##########
@@ -45,11 +45,13 @@
         super.beforeTestsStarted();
         cleanPersistenceDir();
 
-        //startGridsMultiThreaded(1);
         startGrid(0);
         startGrid(1);
+
         grid(0).cluster().state(ClusterState.ACTIVE);
 
+        //Thread.sleep(500);

Review comment:
       Done




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -163,61 +179,68 @@ else if (db == null)
                 if (log.isInfoEnabled())
                     log.info(String.format("Statistics usage state was changed from %s to %s", oldVal, newVal));
 
+                lastUsageState = newVal;
+
                 if (oldVal == newVal)
                     return;
 
-                switch (newVal) {
-                    case OFF:
-                        disableOperations();
-
-                        break;
-                    case ON:
-                    case NO_UPDATE:
-                        enableOperations();
-
-                        break;
-                }
+                stateChanged();

Review comment:
       done, but not start/stop - tryStart/tryStop because there are a lot of conditions to be meet to start.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+

Review comment:
       between two blocks if and if. Or what are you mean?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
##########
@@ -203,11 +239,43 @@ public IgniteStatisticsInMemoryStoreImpl(Function<Class<?>, IgniteLogger> logSup
         Collection<ObjectPartitionStatisticsImpl> statistics
     ) {
         IntMap<ObjectPartitionStatisticsImpl> statisticsMap = new IntHashMap<ObjectPartitionStatisticsImpl>();
+
         for (ObjectPartitionStatisticsImpl s : statistics) {
             if (statisticsMap.put(s.partId(), s) != null)
                 log.warning(String.format("Trying to save more than one %s.%s partition statistics for partition %d",
                     key.schema(), key.obj(), s.partId()));
         }
+
         return statisticsMap;
     }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadObsolescenceMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();

Review comment:
       removed.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -375,6 +292,15 @@ public ObjectStatisticsImpl getLocalStatistics(StatisticsKey key) {
         return locStats.get(key);
     }
 
+    /**
+     * Get all local statistics. Return internal map without copying.
+     *
+     * @return Local (for current node) object statistics.
+     */
+    public Map<StatisticsKey, ObjectStatisticsImpl> getAllLocalStatisticsInt() {

Review comment:
       done

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
##########
@@ -203,11 +239,43 @@ public IgniteStatisticsInMemoryStoreImpl(Function<Class<?>, IgniteLogger> logSup
         Collection<ObjectPartitionStatisticsImpl> statistics
     ) {
         IntMap<ObjectPartitionStatisticsImpl> statisticsMap = new IntHashMap<ObjectPartitionStatisticsImpl>();
+
         for (ObjectPartitionStatisticsImpl s : statistics) {
             if (statisticsMap.put(s.partId(), s) != null)
                 log.warning(String.format("Trying to save more than one %s.%s partition statistics for partition %d",
                     key.schema(), key.obj(), s.partId()));
         }
+
         return statisticsMap;
     }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadObsolescenceMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();
+
+        obsStats.computeIfPresent(key, (k, v) -> {
+            for (Integer partId : v.keys())
+                res[0].add(partId);
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadLocalPartitionMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();

Review comment:
       removed

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -144,10 +137,10 @@
         }
     };
 
-    /** Exchange listener. */
+    /** Exchange listener to update all local statistics. */
     private final PartitionsExchangeAware exchAwareLsnr = new PartitionsExchangeAware() {
         @Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {

Review comment:
       To not to hold topology. Anyway, we schedule processing asynchronously.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -353,6 +252,7 @@ public ObjectPartitionStatisticsImpl getLocalPartitionStatistics(StatisticsKey k
      */
     public void clearLocalPartitionStatistics(StatisticsKey key, int partId) {

Review comment:
       All other methods use LocalPartition to emphasize that they operate with partition level 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.

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];

Review comment:
       Redundant collections.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -308,16 +316,30 @@ public void start() {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager starting...");
 
-        exchange.registerExchangeAwareComponent(exchAwareLsnr);
-
         schemaMgr.registerDropColumnsListener(dropColsLsnr);
         schemaMgr.registerDropTableListener(dropTblLsnr);
 
         if (log.isDebugEnabled())
             log.debug("Statistics configuration manager started.");
 
         if (distrMetaStorage != null)
-            scanAndCheckLocalStatistics(exchange.readyAffinityVersion());
+            mgmtPool.submit(() -> updateFullCfg());

Review comment:
       Busylock?




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -163,61 +179,68 @@ else if (db == null)
                 if (log.isInfoEnabled())
                     log.info(String.format("Statistics usage state was changed from %s to %s", oldVal, newVal));
 
+                lastUsageState = newVal;
+
                 if (oldVal == newVal)
                     return;
 
-                switch (newVal) {
-                    case OFF:
-                        disableOperations();
-
-                        break;
-                    case ON:
-                    case NO_UPDATE:
-                        enableOperations();
-
-                        break;
-                }
+                stateChanged();

Review comment:
       It would be better to call start/stop directly or pass state into.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+

Review comment:
       empty 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.

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -260,12 +289,14 @@ public IgniteStatisticsRepository statisticsRepository() {
 
     /** {@inheritDoc} */
     @Override public void dropAll() throws IgniteCheckedException {
+        checkStatisticsState("drop all statistics");
+
         statCfgMgr.dropAll();
     }
 
     /** {@inheritDoc} */
     @Override public void stop() {
-        disableOperations();
+        stateChanged();

Review comment:
       StatChanged callback shouldn't called here. A callback should call this (stop) method instead.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))

Review comment:
       String is calculated every time, but logged only in debug.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            if ((v.byObsolescence() && ctx.byObsolescence()) ||
+                (v.topologyVersion().before(ctx.topologyVersion()) ||
+                    !checkStatisticsCfg(v.configuration(), ctx.configuration()))) {
+                // Old context for older topology or config - cancel and start new
+                v.future().cancel(true);
+                ctxToAwait[0] = v;
+
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            ctxToSubmit[0] = null;
+
+            return v;
+        });
+
+        // Can't wait in map critical section (to allow task to try to remove itselves), but
+        // have to wait here in busyLock to do gracefull shutdown.
+        if (ctxToAwait[0] != null) {
+            try {
+                ctxToAwait[0].finished().await();
+            }
+            catch (InterruptedException e) {
+                log.warning("Unable to wait statistics gathering task finished by key " +
+                    ctx.configuration().key(), e);
+            }
+        }
+
+        return ctxToSubmit[0];
+    }
+
+    /**
+     * Test if statistics configuration is fit to all required versions and doesn't
+     * contain any extra column configurations.
+     *
+     * @param existingCfg Statistics configuration to check.
+     * @param targetCfg Target configuration.
+     * @return {@code true} if it is, {@code false} otherwise.
+     */
+    private boolean checkStatisticsCfg(
+        StatisticsObjectConfiguration existingCfg,
+        StatisticsObjectConfiguration targetCfg
+    ) {
+        if (existingCfg == targetCfg)
+            return true;
+
+        if (existingCfg.columns().size() != targetCfg.columns().size())
+            return false;
+
+        for (Map.Entry<String, StatisticsColumnConfiguration> targetColCfgEntry : targetCfg.columns().entrySet()) {
+            StatisticsColumnConfiguration existingColCfg = existingCfg.columns().get(targetColCfgEntry.getKey());
+
+            if (existingColCfg == null || existingColCfg.version() < targetColCfgEntry.getValue().version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(ctx, task);
+        }
+    }
+
+    /**
+     *
+     * @param ctx
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (checkCancelled(ctx))
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.byObsolescence()) {
+            if (ctx.configuration() == null)
+                return;
+
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        }
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionIdsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Check if specified context cancelled. If so - log debug message and return {@code true}.
+     *
+     * @param ctx Gathering context to check.
+     * @return {@code true} if context was cancelled, {@code false} - otherwise.z
+     */
+    private boolean checkCancelled(LocalStatisticsGatheringContext ctx) {
+        if (ctx.future().isCancelled()) {
+            if (log.isDebugEnabled())
+                log.debug("Gathering context by key " + ctx.configuration().key() + " cancelled.");
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @param operation Text description of operation to log.
+     * @return {@code true}
+     */
+    private boolean startJob(String operation) {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+        // Partition skipped
+        ctx.finished().countDown();
+
+        // No need to gather rest partitions.
+        ctx.future().cancel(true);
+
+        if (log.isDebugEnabled())
+            log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(), partId));
+
+        if (ctx.partitionNotAvailable(partId)) {
+            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+
+            assert ctx.finished().getCount() == 0;
+
+            if (log.isDebugEnabled())
+                log.debug(String.format("Gathering removed for key %s", ctx.configuration().key()));
+        }
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param ctx Gathering context to track state.
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task
+    ) {
+        gatherPool.submit(() -> {
+            if (!startJob("Gathering partition statistics by key " + ctx.configuration().key())) {
+                failPartTask(ctx, task.partition());
+
+                return;
+            }
+            try {
+                GatheredPartitionResult gatherRes = new GatheredPartitionResult();
+
+                try {
+                    gatherCtxPartition(ctx, task, gatherRes);
+
+                    completePartitionStatistic(gatherRes.newPart, ctx, task.partition(), gatherRes.partStats);
+                }
+                catch (Throwable t) {
+                    //failPartTask(ctx, task.partition());
+                    gatherRes.partStats = null;
+
+                    if (t instanceof GatherStatisticCancelException) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Collect statistics task was cancelled " +
+                                "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                        }
+                    }
+                    else if (t.getCause() instanceof NodeStoppingException) {
+                        if (log.isDebugEnabled())
+                            log.debug("Node stopping during statistics collection on " +
+                                "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                    else
+                        log.warning("Unexpected error on statistic gathering", t);
+                }
+                finally {
+                    if (gatherRes.partStats == null)
+                        failPartTask(ctx, task.partition());
+                    else {
+                        // Finish partition task
+                        ctx.finished().countDown();
+
+                        if (ctx.partitionDone(task.partition()))
+                            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                    }
+                }
+            }
+            finally {
+                endJob();
+            }
+        });
+    }
+
+    private void gatherCtxPartition(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task,
+        GatheredPartitionResult result
+    ) {
+        if (!ctx.byObsolescence()) {
+            // Try to use existing partition statistics instead of gather new one
+            ObjectPartitionStatisticsImpl existingPartStat = statRepo.getLocalPartitionStatistics(
+                ctx.configuration().key(), task.partition());
+
+            if (existingPartStat != null && partStatFit(existingPartStat, ctx.configuration()))
+                result.partStats = existingPartStat;
+        }
+
+        result.newPart = result.partStats == null;
+
+        if (result.partStats == null)
+            result.partStats = task.call();
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatFit(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Complete gathering of partition statistics: save to repository and try to complete whole task.
+     *
+     * @param newStat If {@code true} - partition statitsics was just gathered and need to be saved to repo.
+     * @param ctx Gathering context.
+     * @param partId Partition id.
+     * @param partStat Collected statistics or {@code null} if it was impossible to gather current partition.
+     */
+    private void completePartitionStatistic(
+        boolean newStat,
+        LocalStatisticsGatheringContext ctx,
+        int partId,
+        ObjectPartitionStatisticsImpl partStat
+    ) {
+        if (ctx.future().isCancelled() || partStat == null)
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+
+        if (newStat) {
+            if (ctx.configuration().columns().size() == partStat.columnsStatistics().size())
+                statRepo.refreshObsolescence(key, partId);
+
+            statRepo.replaceLocalPartitionStatistics(key, partStat);
+            //statRepo.saveLocalPartitionStatistics(key, partStat);

Review comment:
       Commented code




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

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

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9423:
URL: https://github.com/apache/ignite/pull/9423#discussion_r724319407



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();

Review comment:
       What happens when deactivating a task throws an 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.

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/task/GatherPartitionStatistics.java
##########
@@ -97,56 +110,161 @@ public int partition() {
         return partId;
     }
 
-    /** {@inheritDoc} */
+    /**
+     * @return LocalStatisticsGatheringContext.
+     */
+    public LocalStatisticsGatheringContext context() {
+        return gathCtx;
+    }
+
+    /**
+     * Reuse or gather new partition statistics according to context and repository state.
+     * Save partition statistcs and obsolescence info back to repository if needed.

Review comment:
       ```suggestion
        * Save partition statistics and obsolescence info back to repository if needed.
   ```




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,35 +82,33 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
+    /** Subsctiption processor. */
     private final GridInternalSubscriptionProcessor subscriptionProcessor;
 
-    /** */
+    /** Exchange manager. */
     private final GridCachePartitionExchangeManager exchange;
 
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
         @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {

Review comment:
       Does it make sense to do anything if metastorage is readonly?




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -260,12 +289,14 @@ public IgniteStatisticsRepository statisticsRepository() {
 
     /** {@inheritDoc} */
     @Override public void dropAll() throws IgniteCheckedException {
+        checkStatisticsState("drop all statistics");
+
         statCfgMgr.dropAll();
     }
 
     /** {@inheritDoc} */
     @Override public void stop() {
-        disableOperations();
+        stateChanged();

Review comment:
       StatChanged callback shouldn't called here. A callback should call this (stop) method instead.
   Otherwise, it looks like components can be started 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.

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {

Review comment:
       1) Reason to create this class was simplicity - to move taking busy lock and checking working status to some wrapper class.
   2) Lock all works in case of problem in one place - is just how our's BusyLock or traditional ReentrantReadWriteLock works. And no matter, is it used from the same class or from separated BusyExecutor. Moreover, BusyExecuter can guarantee that all task will be wrapped in try/catch block with leaveBusy() at final section.
   3) It definitely won't work in multi JVM environment because it shouldn't - we have no multiJVM nodes.
   Are you sure we need to move BusyLock back to StatisticsManager, GlobalStatisticsManager, StatisticsProcessor?




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,35 +82,33 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
+    /** Subsctiption processor. */
     private final GridInternalSubscriptionProcessor subscriptionProcessor;
 
-    /** */
+    /** Exchange manager. */
     private final GridCachePartitionExchangeManager exchange;
 
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
         @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {

Review comment:
       Does it make sense to do anything if metastore is readonly?




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;

Review comment:
       IgniteStatisticsManagerImpl creates management pool to process events like change state, topology or statistics configuration and schedule tasks to gathering pool.




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {

Review comment:
       What reason to have the class? I'm sure that implicit lock at concrete place will be more understandable and safe. Right now we could lock all works in case problem in one place (thread  take a lock and dead, as example). Or how it will work in multi JVM environment?
   Let's move locks to place where it uses.




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -163,61 +194,88 @@ else if (db == null)
                 if (log.isInfoEnabled())
                     log.info(String.format("Statistics usage state was changed from %s to %s", oldVal, newVal));
 
+                lastUsageState = newVal;
+
                 if (oldVal == newVal)
                     return;
 
-                switch (newVal) {
-                    case OFF:
-                        disableOperations();
+                if (newVal == ON || newVal == NO_UPDATE)
+                    tryStart();
 
-                        break;
-                    case ON:
-                    case NO_UPDATE:
-                        enableOperations();
-
-                        break;
-                }
+                if (newVal == OFF)
+                    tryStop();
             });
 
             dispatcher.registerProperty(usageState);
         });
 
-        StatisticsUsageState currState = usageState();
-        if (currState == ON || currState == NO_UPDATE)
-            enableOperations();
-
-        ctx.timeout().schedule(() -> {
-            StatisticsUsageState state = usageState();
-            if (state == ON && !ctx.isStopping()) {
-                if (log.isTraceEnabled())
-                    log.trace("Processing statistics obsolescence...");
-
-                try {
-                    processObsolescence();
-                } catch (Throwable e) {
-                    log.warning("Error while processing statistics obsolescence", e);
-                }
-            }
+        tryStart();
 
-        }, OBSOLESCENCE_INTERVAL * 1000, OBSOLESCENCE_INTERVAL * 1000);
+        if (!ctx.clientNode()) {
+            // Use mgmt pool to work with statistics repository in busy lock to schedule some tasks.
+            ctx.timeout().schedule(() -> {

Review comment:
       Let's stop scheduled task during the component 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.

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final BusyExecutor gatheringBusyExecutor;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatheringBusyExecutor = new BusyExecutor("gathering", gatherPool, logSupplier);
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     *
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have newer configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.

Review comment:
       ```suggestion
        * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - otherwise.
   ```




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -297,78 +332,104 @@ public IgniteStatisticsConfigurationManager statisticConfiguration() {
 
     /** {@inheritDoc} */
     @Override public StatisticsUsageState usageState() {
-        return usageState.getOrDefault(DEFAULT_STATISTICS_USAGE_STATE);
+        return (lastUsageState == null) ? DEFAULT_STATISTICS_USAGE_STATE : lastUsageState;
     }
 
     /** {@inheritDoc} */
     @Override public void onRowUpdated(String schemaName, String objName, int partId, byte[] keyBytes) {
-        try {
-            if (statCfgMgr.config(new StatisticsKey(schemaName, objName)) != null)
-                statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isInfoEnabled())
-                log.info(String.format("Error while obsolescence key in %s.%s due to %s", schemaName, objName,
-                    e.getMessage()));
-        }
+        statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
     }
 
     /**
      * Save dirty obsolescence info to local metastore. Check if statistics need to be refreshed and schedule it.
+     *
+     * 1) Get all dirty partition statistics.
+     * 2) Make separate tasks for each key to avoid saving obsolescence info for removed partition (race).
+     * 3) Check if partition should be recollected and add it to list in its tables task.
+     * 4) Submit tasks. Actually obsolescence info will be stored during task processing.
      */
     public synchronized void processObsolescence() {
-        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.saveObsolescenceInfo();
+        StatisticsUsageState state = usageState();
 
-        Map<StatisticsKey, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
-
-        if (!F.isEmpty(tasks))
-            if (log.isTraceEnabled())
-                log.trace(String.format("Refreshing statistics for %d targets", tasks.size()));
+        if (state != ON || ctx.isStopping()) {
+            if (log.isDebugEnabled())
+                log.debug("Skipping obsolescence processing.");
 
-        for (Map.Entry<StatisticsKey, List<Integer>> objTask : tasks.entrySet()) {
-            GridH2Table tbl = schemaMgr.dataTable(objTask.getKey().schema(), objTask.getKey().obj());
+            return;
+        }

Review comment:
       The second condition looks ambiguous.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -528,47 +444,69 @@ public void updateObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Intege
 
         for (Map.Entry<StatisticsKey, Set<Integer>> objDeleted : deleted.entrySet())
             store.clearObsolescenceInfo(objDeleted.getKey(), objDeleted.getValue());
+
+        fitObsolescenceInfo(cfg);
     }
 
     /**
-     * Load or update obsolescence info cache to fit specified cfg.
+     * Check store to clean unnecessary records.
      *
      * @param cfg Map object statistics configuration to primary partitions set.
      */
-    public synchronized void checkObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {
-        if (!started.compareAndSet(false, true))
-            loadObsolescenceInfo(cfg);
-        else
-            updateObsolescenceInfo(cfg);
+    private void fitObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {

Review comment:
       Why don't use BitSet instead of collections of boxed Integers.




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

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

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9423:
URL: https://github.com/apache/ignite/pull/9423#discussion_r724322630



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is the point of taking a lock?




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is it?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is it?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final BusyExecutor gatheringBusyExecutor;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatheringBusyExecutor = new BusyExecutor("gathering", gatherPool, logSupplier);
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     *
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have newer configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    boolean rescheduled = gatheringBusyExecutor.busyRun(() -> updateLocalStatistics(ctx));
+
+                    if (!rescheduled && log.isDebugEnabled()) {
+                        log.debug("Unable to reschedule statistics task by key " + ctx.configuration().key()
+                            + " due to inactive state.");

Review comment:
       it's not so, busyRun could return false in case any exception 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.

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -144,10 +137,10 @@
         }
     };
 
-    /** Exchange listener. */
+    /** Exchange listener to update all local statistics. */
     private final PartitionsExchangeAware exchAwareLsnr = new PartitionsExchangeAware() {
         @Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {

Review comment:
       Why not before topology unlocked?




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

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

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



[GitHub] [ignite] tledkov-gridgain merged pull request #9423: IGNITE-15281 New implementation of local statistics collection.

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain merged pull request #9423:
URL: https://github.com/apache/ignite/pull/9423


   


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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            if ((v.byObsolescence() && ctx.byObsolescence()) ||
+                (v.topologyVersion().before(ctx.topologyVersion()) ||
+                    !checkStatisticsCfg(v.configuration(), ctx.configuration()))) {
+                // Old context for older topology or config - cancel and start new
+                v.future().cancel(true);
+                ctxToAwait[0] = v;
+
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            ctxToSubmit[0] = null;
+
+            return v;
+        });
+
+        // Can't wait in map critical section (to allow task to try to remove itselves), but
+        // have to wait here in busyLock to do gracefull shutdown.
+        if (ctxToAwait[0] != null) {
+            try {
+                ctxToAwait[0].finished().await();
+            }
+            catch (InterruptedException e) {
+                log.warning("Unable to wait statistics gathering task finished by key " +
+                    ctx.configuration().key(), e);
+            }
+        }
+
+        return ctxToSubmit[0];
+    }
+
+    /**
+     * Test if statistics configuration is fit to all required versions and doesn't
+     * contain any extra column configurations.
+     *
+     * @param existingCfg Statistics configuration to check.
+     * @param targetCfg Target configuration.
+     * @return {@code true} if it is, {@code false} otherwise.
+     */
+    private boolean checkStatisticsCfg(
+        StatisticsObjectConfiguration existingCfg,
+        StatisticsObjectConfiguration targetCfg
+    ) {
+        if (existingCfg == targetCfg)
+            return true;
+
+        if (existingCfg.columns().size() != targetCfg.columns().size())
+            return false;
+
+        for (Map.Entry<String, StatisticsColumnConfiguration> targetColCfgEntry : targetCfg.columns().entrySet()) {
+            StatisticsColumnConfiguration existingColCfg = existingCfg.columns().get(targetColCfgEntry.getKey());
+
+            if (existingColCfg == null || existingColCfg.version() < targetColCfgEntry.getValue().version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(ctx, task);
+        }
+    }
+
+    /**
+     *
+     * @param ctx
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (checkCancelled(ctx))
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.byObsolescence()) {
+            if (ctx.configuration() == null)
+                return;
+
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        }
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionIdsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Check if specified context cancelled. If so - log debug message and return {@code true}.
+     *
+     * @param ctx Gathering context to check.
+     * @return {@code true} if context was cancelled, {@code false} - otherwise.z
+     */
+    private boolean checkCancelled(LocalStatisticsGatheringContext ctx) {
+        if (ctx.future().isCancelled()) {
+            if (log.isDebugEnabled())
+                log.debug("Gathering context by key " + ctx.configuration().key() + " cancelled.");
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @param operation Text description of operation to log.
+     * @return {@code true}
+     */
+    private boolean startJob(String operation) {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+        // Partition skipped
+        ctx.finished().countDown();
+
+        // No need to gather rest partitions.
+        ctx.future().cancel(true);
+
+        if (log.isDebugEnabled())
+            log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(), partId));
+
+        if (ctx.partitionNotAvailable(partId)) {
+            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+
+            assert ctx.finished().getCount() == 0;
+
+            if (log.isDebugEnabled())
+                log.debug(String.format("Gathering removed for key %s", ctx.configuration().key()));
+        }
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param ctx Gathering context to track state.
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task
+    ) {
+        gatherPool.submit(() -> {
+            if (!startJob("Gathering partition statistics by key " + ctx.configuration().key())) {
+                failPartTask(ctx, task.partition());
+
+                return;
+            }
+            try {
+                GatheredPartitionResult gatherRes = new GatheredPartitionResult();
+
+                try {
+                    gatherCtxPartition(ctx, task, gatherRes);
+
+                    completePartitionStatistic(gatherRes.newPart, ctx, task.partition(), gatherRes.partStats);
+                }
+                catch (Throwable t) {
+                    //failPartTask(ctx, task.partition());

Review comment:
       Commented code.




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final BusyExecutor gatheringBusyExecutor;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatheringBusyExecutor = new BusyExecutor("gathering", gatherPool, logSupplier);
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     *
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have newer configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    boolean rescheduled = gatheringBusyExecutor.busyRun(() -> updateLocalStatistics(ctx));
+
+                    if (!rescheduled && log.isDebugEnabled()) {
+                        log.debug("Unable to reschedule statistics task by key " + ctx.configuration().key()
+                            + " due to inactive state.");

Review comment:
       it's not so, busyRun could return false in case any exception 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.

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final BusyExecutor gatheringBusyExecutor;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatheringBusyExecutor = new BusyExecutor("gathering", gatherPool, logSupplier);
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     *
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have newer configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    boolean rescheduled = gatheringBusyExecutor.busyRun(() -> updateLocalStatistics(ctx));
+
+                    if (!rescheduled && log.isDebugEnabled()) {
+                        log.debug("Unable to reschedule statistics task by key " + ctx.configuration().key()
+                            + " due to inactive state.");

Review comment:
       Yes, but updateLocalStatistics never fire exception by itself.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       Just to wait till each running task stops.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,345 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final BusyExecutor gatheringBusyExecutor;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatheringBusyExecutor = new BusyExecutor("gathering", gatherPool, logSupplier);
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     *
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have newer configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.

Review comment:
       Fixed

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();

Review comment:
       Exception just will be pass to code which use BusyExecutor and it should handle it (maybe by calling deactivate again, but here we have not enough information about tasks nature and tracking to handle it.




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

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

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



[GitHub] [ignite] tledkov-gridgain commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

Posted by GitBox <gi...@apache.org>.
tledkov-gridgain commented on a change in pull request #9423:
URL: https://github.com/apache/ignite/pull/9423#discussion_r724319407



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();

Review comment:
       What happens when deactivating a task throws an exception?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is the point of taking a lock?




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,115 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
-    private final IgniteThreadPoolExecutor mgmtPool;
+    private final BusyExecutor mgmtPool;

Review comment:
       strange name for BusyExecotor instance




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();

Review comment:
       Replaces with BusyExecutor.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {
+        return gatheringInProgress.get(key);
+    }
+
+    /**
+     * Start gathering.
+     */
+    public void start() {
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering started.");
+
+        active = true;
+    }
+
+    /**
+     * Stop gathering.
+     */
+    public void stop() {
+        if (log.isTraceEnabled())
+            log.trace(String.format("Statistics gathering stopping %d task...", gatheringInProgress.size()));
+
+        active = false;
+
+        cancelAllTasks();
+
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering stopped.");
+    }
+
+    /**
+     * Cancel all currently running statistics gathering tasks.
+     */
+    public void cancelAllTasks() {
+        gatheringInProgress.values().forEach(g -> g.future().cancel(true));
+        // Can skip waiting for each task finished because of global busyLock.
+
+        gatheringInProgress.clear();
+
+        busyLock.block();
+        busyLock.unblock();
+    }
+
+    /**
+     * Get gathering pool.
+     *
+     * @return Gathering pool.
+     */
+    public IgniteThreadPoolExecutor gatheringPool() {

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.

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

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



[GitHub] [ignite] korlov42 commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
##########
@@ -234,6 +234,37 @@ public byte maxPartitionObsolescencePercent() {
         return S.toString(StatisticsObjectConfiguration.class, this);
     }
 
+    /**
+     * Compare only configuration from the same branch. I.e. can't correctly compare
+     * Cfg(A=v1,B=v3) vs Cfg(A=v2,B=v1)
+     * Cfg(A=v1,B=v3) vs Cfg(A=v1m C=v2)
+     * because there is no changes chain to get one from another.
+     *
+     * @param o Other configuration to compare.
+     * @return Comparison result.
+     */
+    @Override public int compareTo(@NotNull StatisticsObjectConfiguration o) {
+        if (this == o)
+            return 0;
+
+        if (cols.size() < o.cols.size())

Review comment:
       many places in this class cannot handle the `null` column map. This one is one of those places

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsStorageAbstractTest.java
##########
@@ -45,11 +45,13 @@
         super.beforeTestsStarted();
         cleanPersistenceDir();
 
-        //startGridsMultiThreaded(1);
         startGrid(0);
         startGrid(1);
+
         grid(0).cluster().state(ClusterState.ACTIVE);
 
+        Thread.sleep(500);

Review comment:
       why do we need this `sleep`?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {
-                        try {
-                            onChangeStatisticConfiguration(
-                                (StatisticsObjectConfiguration)oldV,
-                                (StatisticsObjectConfiguration)newV
-                            );
-                        }
-                        catch (Throwable e) {
-                            log.warning("Unexpected exception on change statistic configuration [old="
-                                + oldV + ", new=" + newV + ']', e);
-                        }
+                        StatisticsObjectConfiguration newStatCfg = (StatisticsObjectConfiguration)newV;
+
+                        statProc.busyRun(() -> updateLocalStatistics(newStatCfg));
+
                     });
                 }
             );
         }
     };
 
-    /** Exchange listener. */
-    private final PartitionsExchangeAware exchAwareLsnr = new PartitionsExchangeAware() {
-        @Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
-            started = true;
-
-            // Skip join/left client nodes.
-            if (fut.exchangeType() != GridDhtPartitionsExchangeFuture.ExchangeType.ALL ||
-                cluster.clusterState().lastState() != ClusterState.ACTIVE)
-                return;
+    /**
+     * Update statistics after topology change, if necessary.
+     *
+     * @param fut Topology change future.
+     */
+    public void afterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {

Review comment:
       please move all methods below constructor

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {

Review comment:
       unused

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
##########
@@ -169,6 +169,9 @@ public boolean collectionAwareEqual(StatisticsColumnConfiguration o) {
      */
     public StatisticsColumnConfiguration createTombstone()
     {

Review comment:
       C-style braces

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
##########
@@ -23,7 +23,7 @@
 import org.apache.ignite.internal.util.collection.IntMap;
 
 /**
- * Statistics store interface.
+ * Statistics store interface. Wrap phisical storage (in memory or local metastore) to give easy access to statistics.

Review comment:
       physical

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();

Review comment:
       why it's static?

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {
+        return gatheringInProgress.get(key);
+    }
+
+    /**
+     * Start gathering.
+     */
+    public void start() {
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering started.");
+
+        active = true;
+    }
+
+    /**
+     * Stop gathering.
+     */
+    public void stop() {
+        if (log.isTraceEnabled())
+            log.trace(String.format("Statistics gathering stopping %d task...", gatheringInProgress.size()));
+
+        active = false;
+
+        cancelAllTasks();
+
+        if (log.isDebugEnabled())
+            log.debug("Statistics gathering stopped.");
+    }
+
+    /**
+     * Cancel all currently running statistics gathering tasks.
+     */
+    public void cancelAllTasks() {
+        gatheringInProgress.values().forEach(g -> g.future().cancel(true));
+        // Can skip waiting for each task finished because of global busyLock.
+
+        gatheringInProgress.clear();
+
+        busyLock.block();
+        busyLock.unblock();
+    }
+
+    /**
+     * Get gathering pool.
+     *
+     * @return Gathering pool.
+     */
+    public IgniteThreadPoolExecutor gatheringPool() {

Review comment:
       unused

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();

Review comment:
       `mux` is not used anymore

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {

Review comment:
       execute




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -260,12 +289,14 @@ public IgniteStatisticsRepository statisticsRepository() {
 
     /** {@inheritDoc} */
     @Override public void dropAll() throws IgniteCheckedException {
+        checkStatisticsState("drop all statistics");
+
         statCfgMgr.dropAll();
     }
 
     /** {@inheritDoc} */
     @Override public void stop() {
-        disableOperations();
+        stateChanged();

Review comment:
       Done.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -365,6 +265,23 @@ public void saveLocalStatistics(StatisticsKey key, ObjectStatisticsImpl statisti
         locStats.put(key, statistics);
     }
 
+    /**
+     * Clear specified partition ids statistics.
+     *
+     * @param key Key to remove statistics by.
+     * @param partsToRemove Set of parititon ids to remove.
+     */
+    public void clearLocalPartitionIdsStatistics(StatisticsKey key, Set<Integer> partsToRemove) {

Review comment:
       Remove Ids, keep Partition. Local statistics also are stored by the StatisticsRepository but these method clears partition statistics.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            if ((v.byObsolescence() && ctx.byObsolescence()) ||
+                (v.topologyVersion().before(ctx.topologyVersion()) ||
+                    !checkStatisticsCfg(v.configuration(), ctx.configuration()))) {
+                // Old context for older topology or config - cancel and start new
+                v.future().cancel(true);
+                ctxToAwait[0] = v;
+
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            ctxToSubmit[0] = null;
+
+            return v;
+        });
+
+        // Can't wait in map critical section (to allow task to try to remove itselves), but
+        // have to wait here in busyLock to do gracefull shutdown.
+        if (ctxToAwait[0] != null) {
+            try {
+                ctxToAwait[0].finished().await();
+            }
+            catch (InterruptedException e) {
+                log.warning("Unable to wait statistics gathering task finished by key " +
+                    ctx.configuration().key(), e);
+            }
+        }
+
+        return ctxToSubmit[0];
+    }
+
+    /**
+     * Test if statistics configuration is fit to all required versions and doesn't
+     * contain any extra column configurations.
+     *
+     * @param existingCfg Statistics configuration to check.
+     * @param targetCfg Target configuration.
+     * @return {@code true} if it is, {@code false} otherwise.
+     */
+    private boolean checkStatisticsCfg(
+        StatisticsObjectConfiguration existingCfg,
+        StatisticsObjectConfiguration targetCfg
+    ) {
+        if (existingCfg == targetCfg)
+            return true;
+
+        if (existingCfg.columns().size() != targetCfg.columns().size())
+            return false;
+
+        for (Map.Entry<String, StatisticsColumnConfiguration> targetColCfgEntry : targetCfg.columns().entrySet()) {
+            StatisticsColumnConfiguration existingColCfg = existingCfg.columns().get(targetColCfgEntry.getKey());
+
+            if (existingColCfg == null || existingColCfg.version() < targetColCfgEntry.getValue().version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(ctx, task);
+        }
+    }
+
+    /**
+     *
+     * @param ctx
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (checkCancelled(ctx))
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.byObsolescence()) {
+            if (ctx.configuration() == null)
+                return;
+
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        }
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionIdsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Check if specified context cancelled. If so - log debug message and return {@code true}.
+     *
+     * @param ctx Gathering context to check.
+     * @return {@code true} if context was cancelled, {@code false} - otherwise.z
+     */
+    private boolean checkCancelled(LocalStatisticsGatheringContext ctx) {
+        if (ctx.future().isCancelled()) {
+            if (log.isDebugEnabled())
+                log.debug("Gathering context by key " + ctx.configuration().key() + " cancelled.");
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @param operation Text description of operation to log.
+     * @return {@code true}
+     */
+    private boolean startJob(String operation) {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+        // Partition skipped
+        ctx.finished().countDown();
+
+        // No need to gather rest partitions.
+        ctx.future().cancel(true);
+
+        if (log.isDebugEnabled())
+            log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(), partId));
+
+        if (ctx.partitionNotAvailable(partId)) {
+            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+
+            assert ctx.finished().getCount() == 0;
+
+            if (log.isDebugEnabled())
+                log.debug(String.format("Gathering removed for key %s", ctx.configuration().key()));
+        }
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param ctx Gathering context to track state.
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task
+    ) {
+        gatherPool.submit(() -> {
+            if (!startJob("Gathering partition statistics by key " + ctx.configuration().key())) {
+                failPartTask(ctx, task.partition());
+
+                return;
+            }
+            try {
+                GatheredPartitionResult gatherRes = new GatheredPartitionResult();
+
+                try {
+                    gatherCtxPartition(ctx, task, gatherRes);
+
+                    completePartitionStatistic(gatherRes.newPart, ctx, task.partition(), gatherRes.partStats);
+                }
+                catch (Throwable t) {
+                    //failPartTask(ctx, task.partition());
+                    gatherRes.partStats = null;
+
+                    if (t instanceof GatherStatisticCancelException) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Collect statistics task was cancelled " +
+                                "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                        }
+                    }
+                    else if (t.getCause() instanceof NodeStoppingException) {
+                        if (log.isDebugEnabled())
+                            log.debug("Node stopping during statistics collection on " +
+                                "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                    else
+                        log.warning("Unexpected error on statistic gathering", t);
+                }
+                finally {
+                    if (gatherRes.partStats == null)
+                        failPartTask(ctx, task.partition());
+                    else {
+                        // Finish partition task
+                        ctx.finished().countDown();
+
+                        if (ctx.partitionDone(task.partition()))
+                            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                    }
+                }
+            }
+            finally {
+                endJob();
+            }
+        });
+    }
+
+    private void gatherCtxPartition(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task,
+        GatheredPartitionResult result
+    ) {
+        if (!ctx.byObsolescence()) {
+            // Try to use existing partition statistics instead of gather new one
+            ObjectPartitionStatisticsImpl existingPartStat = statRepo.getLocalPartitionStatistics(
+                ctx.configuration().key(), task.partition());
+
+            if (existingPartStat != null && partStatFit(existingPartStat, ctx.configuration()))
+                result.partStats = existingPartStat;
+        }
+
+        result.newPart = result.partStats == null;
+
+        if (result.partStats == null)
+            result.partStats = task.call();
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatFit(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Complete gathering of partition statistics: save to repository and try to complete whole task.
+     *
+     * @param newStat If {@code true} - partition statitsics was just gathered and need to be saved to repo.
+     * @param ctx Gathering context.
+     * @param partId Partition id.
+     * @param partStat Collected statistics or {@code null} if it was impossible to gather current partition.
+     */
+    private void completePartitionStatistic(
+        boolean newStat,
+        LocalStatisticsGatheringContext ctx,
+        int partId,
+        ObjectPartitionStatisticsImpl partStat
+    ) {
+        if (ctx.future().isCancelled() || partStat == null)
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+
+        if (newStat) {
+            if (ctx.configuration().columns().size() == partStat.columnsStatistics().size())
+                statRepo.refreshObsolescence(key, partId);
+
+            statRepo.replaceLocalPartitionStatistics(key, partStat);
+            //statRepo.saveLocalPartitionStatistics(key, partStat);

Review comment:
       Removed

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -400,10 +456,21 @@ public synchronized void processObsolescence() {
                     taskParts.add(k);
             });
 
-            if (!taskParts.isEmpty())
-                res.put(key, taskParts);
+            // Will add even empty list of partitions to recollect just to force obsolescence info to be stored.
+            res.put(finalCfg, taskParts);
         }
 
         return res;
     }
+
+    /**
+     * Check that cluster statistics usage state is not OFF and cluster is active.
+     *
+     * @param op Operation name.
+     */
+    public void checkStatisticsState(String op) {

Review comment:
       Done

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))

Review comment:
       Removed.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -297,78 +332,104 @@ public IgniteStatisticsConfigurationManager statisticConfiguration() {
 
     /** {@inheritDoc} */
     @Override public StatisticsUsageState usageState() {
-        return usageState.getOrDefault(DEFAULT_STATISTICS_USAGE_STATE);
+        return (lastUsageState == null) ? DEFAULT_STATISTICS_USAGE_STATE : lastUsageState;
     }
 
     /** {@inheritDoc} */
     @Override public void onRowUpdated(String schemaName, String objName, int partId, byte[] keyBytes) {
-        try {
-            if (statCfgMgr.config(new StatisticsKey(schemaName, objName)) != null)
-                statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isInfoEnabled())
-                log.info(String.format("Error while obsolescence key in %s.%s due to %s", schemaName, objName,
-                    e.getMessage()));
-        }
+        statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
     }
 
     /**
      * Save dirty obsolescence info to local metastore. Check if statistics need to be refreshed and schedule it.
+     *
+     * 1) Get all dirty partition statistics.
+     * 2) Make separate tasks for each key to avoid saving obsolescence info for removed partition (race).
+     * 3) Check if partition should be recollected and add it to list in its tables task.
+     * 4) Submit tasks. Actually obsolescence info will be stored during task processing.
      */
     public synchronized void processObsolescence() {
-        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.saveObsolescenceInfo();
+        StatisticsUsageState state = usageState();
 
-        Map<StatisticsKey, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
-
-        if (!F.isEmpty(tasks))
-            if (log.isTraceEnabled())
-                log.trace(String.format("Refreshing statistics for %d targets", tasks.size()));
+        if (state != ON || ctx.isStopping()) {
+            if (log.isDebugEnabled())
+                log.debug("Skipping obsolescence processing.");
 
-        for (Map.Entry<StatisticsKey, List<Integer>> objTask : tasks.entrySet()) {
-            GridH2Table tbl = schemaMgr.dataTable(objTask.getKey().schema(), objTask.getKey().obj());
+            return;
+        }
 
-            if (tbl == null) {
-                if (log.isDebugEnabled())
-                    log.debug(String.format("Got obsolescence statistics for unknown table %s", objTask.getKey()));
+        if (log.isTraceEnabled())
+            log.trace("Process statistics obsolescence started.");
 
-                continue;
-            }
+        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.getDirtyObsolescenceInfo();
 
-            StatisticsObjectConfiguration objCfg;
-            try {
-                objCfg = statCfgMgr.config(objTask.getKey());
-            } catch (IgniteCheckedException e) {
-                log.warning("Unable to load statistics object configuration from global metastore", e);
-                continue;
-            }
+        if (F.isEmpty(dirty)) {
+            if (log.isTraceEnabled())
+                log.trace("No dirty obsolescence info found. Finish obsolescence processing.");
 
-            if (objCfg == null) {
-                if (log.isDebugEnabled())
-                    log.debug(String.format("Got obsolescence statistics for unknown configuration %s", objTask.getKey()));
+            return;
+        }
+        else {
+            if (log.isTraceEnabled())
+                log.trace(String.format("Scheduling obsolescence savings for %d targets", dirty.size()));
+        }
 
-                continue;
-            }
+        Map<StatisticsObjectConfiguration, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
 
-            GridCacheContext cctx = tbl.cacheContext();
+        for (Map.Entry<StatisticsObjectConfiguration, List<Integer>> objTask : tasks.entrySet()) {
+            StatisticsKey key = objTask.getKey().key();
+            GridH2Table tbl = schemaMgr.dataTable(key.schema(), key.obj());
 
-            Set<Integer> parts = cctx.affinity().primaryPartitions(
-                cctx.localNodeId(), cctx.affinity().affinityTopologyVersion());
+            if (tbl == null) {
+                // Table can be removed earlier, but not already processed. Or somethink goes wrong. Try to reschedule.
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Got obsolescence statistics for unknown table %s", objTask.getKey()));
+            }
 
-            statCfgMgr.gatherLocalStatistics(objCfg, tbl, parts, new HashSet<>(objTask.getValue()), null);
+            statProc.updateKeyAsync(true, tbl, objTask.getKey(), new HashSet<>(objTask.getValue()),
+                null);
+//            if (objTask.getValue().isEmpty()) {
+//                // Just save or totally remove obsolescence info, no additional operations needed.
+//                statProc.updateKeyAsync(true, tbl, objTask.getKey(), Collections.emptySet(), null);
+//            }
+//            else {
+//                // Schedule full gathering.
+//                GridCacheContext<?, ?> cctx = (tbl == null) ? null : tbl.cacheContext();
+//
+//                AffinityTopologyVersion topVer = null;
+//
+//                if (!cctx.gate().enterIfNotStopped())
+//                    continue;
+//
+//                try {
+//                    topVer = cctx.affinity().affinityTopologyVersion();
+//                    cctx.affinity().affinityReadyFuture(topVer).get();
+//                }
+//                catch (IgniteCheckedException e) {
+//                    log.warning("Unable to get topology ready.", e);
+//                }
+//                finally {
+//                    cctx.gate().leave();
+//                }
+//
+//                statProc.updateKeyAsync(true, tbl, objTask.getKey(), new HashSet<>(objTask.getValue()),
+//                    topVer);
+//
+//            }

Review comment:
       Removed

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];

Review comment:
       Rewriten to boolean result.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsManagerImpl.java
##########
@@ -297,78 +332,104 @@ public IgniteStatisticsConfigurationManager statisticConfiguration() {
 
     /** {@inheritDoc} */
     @Override public StatisticsUsageState usageState() {
-        return usageState.getOrDefault(DEFAULT_STATISTICS_USAGE_STATE);
+        return (lastUsageState == null) ? DEFAULT_STATISTICS_USAGE_STATE : lastUsageState;
     }
 
     /** {@inheritDoc} */
     @Override public void onRowUpdated(String schemaName, String objName, int partId, byte[] keyBytes) {
-        try {
-            if (statCfgMgr.config(new StatisticsKey(schemaName, objName)) != null)
-                statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isInfoEnabled())
-                log.info(String.format("Error while obsolescence key in %s.%s due to %s", schemaName, objName,
-                    e.getMessage()));
-        }
+        statsRepos.addRowsModified(new StatisticsKey(schemaName, objName), partId, keyBytes);
     }
 
     /**
      * Save dirty obsolescence info to local metastore. Check if statistics need to be refreshed and schedule it.
+     *
+     * 1) Get all dirty partition statistics.
+     * 2) Make separate tasks for each key to avoid saving obsolescence info for removed partition (race).
+     * 3) Check if partition should be recollected and add it to list in its tables task.
+     * 4) Submit tasks. Actually obsolescence info will be stored during task processing.
      */
     public synchronized void processObsolescence() {
-        Map<StatisticsKey, IntMap<ObjectPartitionStatisticsObsolescence>> dirty = statsRepos.saveObsolescenceInfo();
+        StatisticsUsageState state = usageState();
 
-        Map<StatisticsKey, List<Integer>> tasks = calculateObsolescenceRefreshTasks(dirty);
-
-        if (!F.isEmpty(tasks))
-            if (log.isTraceEnabled())
-                log.trace(String.format("Refreshing statistics for %d targets", tasks.size()));
+        if (state != ON || ctx.isStopping()) {
+            if (log.isDebugEnabled())
+                log.debug("Skipping obsolescence processing.");
 
-        for (Map.Entry<StatisticsKey, List<Integer>> objTask : tasks.entrySet()) {
-            GridH2Table tbl = schemaMgr.dataTable(objTask.getKey().schema(), objTask.getKey().obj());
+            return;
+        }

Review comment:
       Context switches state to stopping without notification.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,615 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /* Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param byObsolescence Update only obsolescence partitions.
+     * @param tbl Table to update. If {@code null} - just clear all partitions and obsolescence from the repo
+     * @param cfg Statistics configuration to use.
+     * @param partsToProcess Partitions to update, if !byObsolescence - all primary partitions for the given topology.
+     * @param topVer Topology version, can be {@code null} if tbl is null.
+     */
+    public void updateKeyAsync(
+        boolean byObsolescence,
+        GridH2Table tbl,
+        StatisticsObjectConfiguration cfg,
+        Set<Integer> partsToProcess,
+        AffinityTopologyVersion topVer
+    ) {
+        if (!startJob("Updating key " + cfg.key()))
+            return;
+
+        try {
+            if (log.isDebugEnabled()) {
+                log.debug(String.format(
+                    "Start statistics processing: byObsolescence=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                    byObsolescence, cfg, partsToProcess, topVer));
+            }
+
+            LocalStatisticsGatheringContext newCtx = new LocalStatisticsGatheringContext(byObsolescence, tbl, cfg,
+                partsToProcess, topVer);
+            LocalStatisticsGatheringContext registeredCtx = registerNewTask(newCtx);
+
+            if (registeredCtx != null) {
+
+                prepareTask(registeredCtx);
+
+                if (registeredCtx.table() != null && !registeredCtx.remainingParts().isEmpty() &&
+                    !registeredCtx.configuration().columns().isEmpty())
+                    submitTasks(registeredCtx);
+                else {
+                    gatheringInProgress.remove(registeredCtx.configuration().key(), registeredCtx);
+
+                    assert registeredCtx.remainingParts().isEmpty();
+                    assert registeredCtx.finished().getCount() == 0;
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Gathered by key " + cfg.key() + " were skipped due to previous one.");
+            }
+        }
+        finally {
+            endJob();
+        }
+    }
+
+    /**
+     * Do preparation step before schedule any gathering.
+     *
+     * @param ctx Context to do preparations.
+     */
+    private void prepareTask(LocalStatisticsGatheringContext ctx) {
+        try {
+            if (ctx.byObsolescence())
+                statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+            if (ctx.table() == null || ctx.configuration().columns().isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+        }
+        catch (Throwable t) {
+            ctx.future().cancel(true);
+        }
+        finally {
+            // Prepare phase done
+            ctx.finished().countDown();
+        }
+    }
+
+    /**
+     * Try to register new task.
+     *
+     * @param ctx Task to register.
+     * @return Registered task.
+     */
+    private LocalStatisticsGatheringContext registerNewTask(LocalStatisticsGatheringContext ctx) {
+        LocalStatisticsGatheringContext ctxToSubmit[] = new LocalStatisticsGatheringContext[1];
+        LocalStatisticsGatheringContext ctxToAwait[] = new LocalStatisticsGatheringContext[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            if ((v.byObsolescence() && ctx.byObsolescence()) ||
+                (v.topologyVersion().before(ctx.topologyVersion()) ||
+                    !checkStatisticsCfg(v.configuration(), ctx.configuration()))) {
+                // Old context for older topology or config - cancel and start new
+                v.future().cancel(true);
+                ctxToAwait[0] = v;
+
+                ctxToSubmit[0] = ctx;
+
+                return ctx;
+            }
+
+            ctxToSubmit[0] = null;
+
+            return v;
+        });
+
+        // Can't wait in map critical section (to allow task to try to remove itselves), but
+        // have to wait here in busyLock to do gracefull shutdown.
+        if (ctxToAwait[0] != null) {
+            try {
+                ctxToAwait[0].finished().await();
+            }
+            catch (InterruptedException e) {
+                log.warning("Unable to wait statistics gathering task finished by key " +
+                    ctx.configuration().key(), e);
+            }
+        }
+
+        return ctxToSubmit[0];
+    }
+
+    /**
+     * Test if statistics configuration is fit to all required versions and doesn't
+     * contain any extra column configurations.
+     *
+     * @param existingCfg Statistics configuration to check.
+     * @param targetCfg Target configuration.
+     * @return {@code true} if it is, {@code false} otherwise.
+     */
+    private boolean checkStatisticsCfg(
+        StatisticsObjectConfiguration existingCfg,
+        StatisticsObjectConfiguration targetCfg
+    ) {
+        if (existingCfg == targetCfg)
+            return true;
+
+        if (existingCfg.columns().size() != targetCfg.columns().size())
+            return false;
+
+        for (Map.Entry<String, StatisticsColumnConfiguration> targetColCfgEntry : targetCfg.columns().entrySet()) {
+            StatisticsColumnConfiguration existingColCfg = existingCfg.columns().get(targetColCfgEntry.getKey());
+
+            if (existingColCfg == null || existingColCfg.version() < targetColCfgEntry.getValue().version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(ctx, task);
+        }
+    }
+
+    /**
+     *
+     * @param ctx
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (checkCancelled(ctx))
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.byObsolescence()) {
+            if (ctx.configuration() == null)
+                return;
+
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        }
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionIdsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Check if specified context cancelled. If so - log debug message and return {@code true}.
+     *
+     * @param ctx Gathering context to check.
+     * @return {@code true} if context was cancelled, {@code false} - otherwise.z
+     */
+    private boolean checkCancelled(LocalStatisticsGatheringContext ctx) {
+        if (ctx.future().isCancelled()) {
+            if (log.isDebugEnabled())
+                log.debug("Gathering context by key " + ctx.configuration().key() + " cancelled.");
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @param operation Text description of operation to log.
+     * @return {@code true}
+     */
+    private boolean startJob(String operation) {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start " + operation + " due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+        // Partition skipped
+        ctx.finished().countDown();
+
+        // No need to gather rest partitions.
+        ctx.future().cancel(true);
+
+        if (log.isDebugEnabled())
+            log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(), partId));
+
+        if (ctx.partitionNotAvailable(partId)) {
+            gatheringInProgress.remove(ctx.configuration().key(), ctx);
+
+            assert ctx.finished().getCount() == 0;
+
+            if (log.isDebugEnabled())
+                log.debug(String.format("Gathering removed for key %s", ctx.configuration().key()));
+        }
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param ctx Gathering context to track state.
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(
+        final LocalStatisticsGatheringContext ctx,
+        final GatherPartitionStatistics task
+    ) {
+        gatherPool.submit(() -> {
+            if (!startJob("Gathering partition statistics by key " + ctx.configuration().key())) {
+                failPartTask(ctx, task.partition());
+
+                return;
+            }
+            try {
+                GatheredPartitionResult gatherRes = new GatheredPartitionResult();
+
+                try {
+                    gatherCtxPartition(ctx, task, gatherRes);
+
+                    completePartitionStatistic(gatherRes.newPart, ctx, task.partition(), gatherRes.partStats);
+                }
+                catch (Throwable t) {
+                    //failPartTask(ctx, task.partition());

Review comment:
       Removed

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsRepository.java
##########
@@ -528,47 +444,69 @@ public void updateObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Intege
 
         for (Map.Entry<StatisticsKey, Set<Integer>> objDeleted : deleted.entrySet())
             store.clearObsolescenceInfo(objDeleted.getKey(), objDeleted.getValue());
+
+        fitObsolescenceInfo(cfg);
     }
 
     /**
-     * Load or update obsolescence info cache to fit specified cfg.
+     * Check store to clean unnecessary records.
      *
      * @param cfg Map object statistics configuration to primary partitions set.
      */
-    public synchronized void checkObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {
-        if (!started.compareAndSet(false, true))
-            loadObsolescenceInfo(cfg);
-        else
-            updateObsolescenceInfo(cfg);
+    private void fitObsolescenceInfo(Map<StatisticsObjectConfiguration, Set<Integer>> cfg) {

Review comment:
       Yes, it's better to choose an optimal data structure. Removed 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.

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -308,16 +316,30 @@ public void start() {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager starting...");
 
-        exchange.registerExchangeAwareComponent(exchAwareLsnr);
-
         schemaMgr.registerDropColumnsListener(dropColsLsnr);
         schemaMgr.registerDropTableListener(dropTblLsnr);
 
         if (log.isDebugEnabled())
             log.debug("Statistics configuration manager started.");
 
         if (distrMetaStorage != null)
-            scanAndCheckLocalStatistics(exchange.readyAffinityVersion());
+            mgmtPool.submit(() -> updateFullCfg());

Review comment:
       yep, 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.

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -308,16 +316,30 @@ public void start() {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager starting...");
 
-        exchange.registerExchangeAwareComponent(exchAwareLsnr);
-
         schemaMgr.registerDropColumnsListener(dropColsLsnr);
         schemaMgr.registerDropTableListener(dropTblLsnr);
 
         if (log.isDebugEnabled())
             log.debug("Statistics configuration manager started.");
 
         if (distrMetaStorage != null)
-            scanAndCheckLocalStatistics(exchange.readyAffinityVersion());
+            mgmtPool.submit(() -> updateFullCfg());

Review comment:
       yep, 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.

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsInMemoryStoreImpl.java
##########
@@ -203,11 +239,43 @@ public IgniteStatisticsInMemoryStoreImpl(Function<Class<?>, IgniteLogger> logSup
         Collection<ObjectPartitionStatisticsImpl> statistics
     ) {
         IntMap<ObjectPartitionStatisticsImpl> statisticsMap = new IntHashMap<ObjectPartitionStatisticsImpl>();
+
         for (ObjectPartitionStatisticsImpl s : statistics) {
             if (statisticsMap.put(s.partId(), s) != null)
                 log.warning(String.format("Trying to save more than one %s.%s partition statistics for partition %d",
                     key.schema(), key.obj(), s.partId()));
         }
+
         return statisticsMap;
     }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadObsolescenceMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();
+
+        obsStats.computeIfPresent(key, (k, v) -> {
+            for (Integer partId : v.keys())
+                res[0].add(partId);
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<Integer> loadLocalPartitionMap(StatisticsKey key) {
+        Collection<Integer> res[] = new Collection[1];
+        res[0] = new ArrayList<>();

Review comment:
       Redundant collection.




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

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

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



[GitHub] [ignite] AMashenkov commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -308,16 +316,30 @@ public void start() {
         if (log.isTraceEnabled())
             log.trace("Statistics configuration manager starting...");
 
-        exchange.registerExchangeAwareComponent(exchAwareLsnr);
-
         schemaMgr.registerDropColumnsListener(dropColsLsnr);
         schemaMgr.registerDropTableListener(dropTblLsnr);
 
         if (log.isDebugEnabled())
             log.debug("Statistics configuration manager started.");
 
         if (distrMetaStorage != null)
-            scanAndCheckLocalStatistics(exchange.readyAffinityVersion());
+            mgmtPool.submit(() -> updateFullCfg());

Review comment:
       Missed busylock?




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/BusyExecutor.java
##########
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Executor with busy run support.
+ * Can run any tasks while active and safelly wait untill they stopped.
+ */
+public class BusyExecutor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Executor name. */
+    private final String name;
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private final GridBusyLock busyLock = new GridBusyLock();
+
+    /** Executor pool. */
+    private final IgniteThreadPoolExecutor pool;
+
+    /**
+     * Constructor.
+     *
+     * @param name Executor name.
+     * @param pool Underlying thread pool executor.
+     * @param logSupplier Log supplier.
+     */
+    public BusyExecutor(String name, IgniteThreadPoolExecutor pool, Function<Class<?>, IgniteLogger> logSupplier) {
+        this.name = name;
+        this.pool = pool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Allow operations.
+     */
+    public void activate() {
+        active = true;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " activated.");
+    }
+
+    /**
+     * Stop all running tasks. Block new task scheduling, execute cancell runnable and wait till each task stops.
+     *
+     * @param r Runnable to cancel all scheduled tasks.
+     */
+    public void deactivate(Runnable r) {
+        active = false;
+
+        if (log.isDebugEnabled())
+            log.debug("Busy executor " + name + " deactivating.");
+
+        r.run();
+
+        busyLock.block();
+        busyLock.unblock();

Review comment:
       What is it?




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

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

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



[GitHub] [ignite] ygerzhedovich commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsStorageAbstractTest.java
##########
@@ -45,11 +45,13 @@
         super.beforeTestsStarted();
         cleanPersistenceDir();
 
-        //startGridsMultiThreaded(1);
         startGrid(0);
         startGrid(1);
+
         grid(0).cluster().state(ClusterState.ACTIVE);
 
+        //Thread.sleep(500);

Review comment:
       no comments - should be deleted




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

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

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



[GitHub] [ignite] Berkof commented on a change in pull request #9423: IGNITE-15281 New implementation of local statistics collection.

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



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsColumnConfiguration.java
##########
@@ -169,6 +169,9 @@ public boolean collectionAwareEqual(StatisticsColumnConfiguration o) {
      */
     public StatisticsColumnConfiguration createTombstone()
     {

Review comment:
       Done.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/query/stat/config/StatisticsObjectConfiguration.java
##########
@@ -234,6 +234,37 @@ public byte maxPartitionObsolescencePercent() {
         return S.toString(StatisticsObjectConfiguration.class, this);
     }
 
+    /**
+     * Compare only configuration from the same branch. I.e. can't correctly compare
+     * Cfg(A=v1,B=v3) vs Cfg(A=v2,B=v1)
+     * Cfg(A=v1,B=v3) vs Cfg(A=v1m C=v2)
+     * because there is no changes chain to get one from another.
+     *
+     * @param o Other configuration to compare.
+     * @return Comparison result.
+     */
+    @Override public int compareTo(@NotNull StatisticsObjectConfiguration o) {
+        if (this == o)
+            return 0;
+
+        if (cols.size() < o.cols.size())

Review comment:
       It cannot be null anymore.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {

Review comment:
       Fixed.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();

Review comment:
       Removed.

##########
File path: modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/stat/StatisticsStorageAbstractTest.java
##########
@@ -45,11 +45,13 @@
         super.beforeTestsStarted();
         cleanPersistenceDir();
 
-        //startGridsMultiThreaded(1);
         startGrid(0);
         startGrid(1);
+
         grid(0).cluster().state(ClusterState.ACTIVE);
 
+        Thread.sleep(500);

Review comment:
       Removed.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsStore.java
##########
@@ -23,7 +23,7 @@
 import org.apache.ignite.internal.util.collection.IntMap;
 
 /**
- * Statistics store interface.
+ * Statistics store interface. Wrap phisical storage (in memory or local metastore) to give easy access to statistics.

Review comment:
       Fixed.

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/IgniteStatisticsConfigurationManager.java
##########
@@ -86,89 +79,82 @@
     /** Distributed metastore. */
     private volatile DistributedMetaStorage distrMetaStorage;
 
-    /** Statistics repository.*/
-    private final IgniteStatisticsRepository repo;
-
-    /** Statistic gatherer. */
-    private final StatisticsGatherer gatherer;
+    /** Statistic processor. */
+    private final StatisticsProcessor statProc;
 
     /** */
     private final IgniteThreadPoolExecutor mgmtPool;
 
+    /** Persistence enabled flag. */
+    private final boolean persistence;
+
     /** Logger. */
     private final IgniteLogger log;
 
-    /** Started flag (used to skip updates of the distributed metastorage on start). */
-    private volatile boolean started;
+    /** Last ready topology version if {@code null} - used to skip updates of the distributed metastorage on start. */
+    private volatile AffinityTopologyVersion topVer;
 
     /** Monitor to synchronize changes repository: aggregate after collects and drop statistics. */
     private final Object mux = new Object();
 
-    /** */
+    /** Cluster state processor. */
     private final GridClusterStateProcessor cluster;
 
-    /** */
-    private final GridInternalSubscriptionProcessor subscriptionProcessor;
-
-    /** */
-    private final GridCachePartitionExchangeManager exchange;
-
-    /** */
-    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr = new DistributedMetastorageLifecycleListener() {
-        @Override public void onReadyForRead(ReadableDistributedMetaStorage metastorage) {
-            distrMetaStorage = (DistributedMetaStorage)metastorage;
+    /** Change statistics configuration listener to update particular object statistics. */
+    private final DistributedMetastorageLifecycleListener distrMetaStoreLsnr =
+        new DistributedMetastorageLifecycleListener() {
+        @Override public void onReadyForWrite(DistributedMetaStorage metastorage) {
+            distrMetaStorage = metastorage;
 
             distrMetaStorage.listen(
                 (metaKey) -> metaKey.startsWith(STAT_OBJ_PREFIX),
                 (k, oldV, newV) -> {
                     // Skip invoke on start node (see 'ReadableDistributedMetaStorage#listen' the second case)
                     // The update statistics on start node is handled by 'scanAndCheckLocalStatistic' method
                     // called on exchange done.
-                    if (!started)
+                    if (topVer == null)
                         return;
 
                     mgmtPool.submit(() -> {
-                        try {
-                            onChangeStatisticConfiguration(
-                                (StatisticsObjectConfiguration)oldV,
-                                (StatisticsObjectConfiguration)newV
-                            );
-                        }
-                        catch (Throwable e) {
-                            log.warning("Unexpected exception on change statistic configuration [old="
-                                + oldV + ", new=" + newV + ']', e);
-                        }
+                        StatisticsObjectConfiguration newStatCfg = (StatisticsObjectConfiguration)newV;
+
+                        statProc.busyRun(() -> updateLocalStatistics(newStatCfg));
+
                     });
                 }
             );
         }
     };
 
-    /** Exchange listener. */
-    private final PartitionsExchangeAware exchAwareLsnr = new PartitionsExchangeAware() {
-        @Override public void onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
-            started = true;
-
-            // Skip join/left client nodes.
-            if (fut.exchangeType() != GridDhtPartitionsExchangeFuture.ExchangeType.ALL ||
-                cluster.clusterState().lastState() != ClusterState.ACTIVE)
-                return;
+    /**
+     * Update statistics after topology change, if necessary.
+     *
+     * @param fut Topology change future.
+     */
+    public void afterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {

Review comment:
       Done

##########
File path: modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/stat/StatisticsProcessor.java
##########
@@ -0,0 +1,453 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY 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.query.stat;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.NodeStoppingException;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsColumnConfiguration;
+import org.apache.ignite.internal.processors.query.stat.config.StatisticsObjectConfiguration;
+import org.apache.ignite.internal.processors.query.stat.task.GatherPartitionStatistics;
+import org.apache.ignite.internal.util.GridBusyLock;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
+
+/**
+ * Process all tasks, related to statistics repository. Mostly - statistics collection,
+ * invalidation (due to configuration, topology or obsolescence issues) and loads.
+ * Input tasks should be scheduled throug management pool while gathering pool used to process heavy
+ * operations in parallel.
+ *
+ * Manage gathering pool and it's jobs. To guarantee gracefull shutdown:
+ * 1) Any job can be added into gatheringInProgress only in active state (check after adding)
+ * 2) State can be disactivated only after cancelling all jobs and getting busyLock block
+ * 3) Each job should do it's work in busyLock with periodically checking of it's cancellation status.
+ */
+public class StatisticsProcessor {
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Ignite statistics repository. */
+    private final IgniteStatisticsRepository statRepo;
+
+    /** Ignite Thread pool executor to do statistics collection tasks. */
+    private final IgniteThreadPoolExecutor gatherPool;
+
+    /** (cacheGroupId -> gather context) */
+    private final ConcurrentMap<StatisticsKey, LocalStatisticsGatheringContext> gatheringInProgress =
+        new ConcurrentHashMap<>();
+
+    /** Active flag (used to skip commands in inactive cluster.) */
+    private volatile boolean active;
+
+    /** Lock protection of started gathering during deactivation. */
+    private static final GridBusyLock busyLock = new GridBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param repo IgniteStatisticsRepository.
+     * @param gatherPool Thread pool to gather statistics in.
+     * @param logSupplier Log supplier function.
+     */
+    public StatisticsProcessor(
+        IgniteStatisticsRepository repo,
+        IgniteThreadPoolExecutor gatherPool,
+        Function<Class<?>, IgniteLogger> logSupplier
+    ) {
+        this.statRepo = repo;
+        this.gatherPool = gatherPool;
+        this.log = logSupplier.apply(StatisticsProcessor.class);
+    }
+
+    /**
+     * Update statistics for the given key to actual state.
+     * If byObsolescence and tbl is not {@code null} - does not clear any other partitions.
+     * Should run throw management pool only.
+     * 1) Replace previous gathering context if exist and needed (replace byObsolescence gathering with new one or
+     * replace gathering with older configuration or topology version with new one).
+     * 2) If byObsolescence and no table awailable - clean obsolescence and partition statistics for the given key.
+     * 3) Submit tasks for each specified partition.
+     * 4) after last task finish gathering - it starts aggregation.
+     * 5) read all partitions & obsolescence from repo and
+     * if byObsolescence = {@code true} - remove unnecessary one and aggregate by specified list
+     * if byObsolescence = {@code false} - aggregate all presented in store (because it should contains only actual ones)
+     * 5) save aggregated local statistics
+     *
+     * @param ctx Statistics Gathering context.
+     */
+    public void updateLocalStatistics(LocalStatisticsGatheringContext ctx) {
+        if (log.isDebugEnabled()) {
+            log.debug(String.format(
+                "Start statistics processing: forceRecollect=%b, cfg=%s, partToProcess = %s, topVer=%s",
+                ctx.forceRecollect(), ctx.configuration(), ctx.allParts(), ctx.topologyVersion()));
+        }
+
+        if (registerNewTask(ctx)) {
+            try {
+                if (ctx.forceRecollect())
+                    // To save all obsolescence info even if there is no partitions to recollect.
+                    statRepo.saveObsolescenceInfo(ctx.configuration().key());
+
+                if (ctx.table() == null || ctx.configuration() == null || ctx.configuration().columns().isEmpty()) {
+                    statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), null);
+                    ctx.future().complete(null);
+
+                    return;
+                }
+
+                if (ctx.remainingParts().isEmpty())
+                    ctx.future().complete(null);
+                else
+                    submitTasks(ctx);
+            }
+            catch (Throwable t) {
+                // Submit tasks can't fire an error, so no need to cancel and wait for tasks finished here.
+                ctx.future().completeExceptionally(t);
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Gathered by key " + ctx.configuration().key() + " were skipped due to previous one.");
+        }
+    }
+
+    /**
+     * Try to register new task. Returned task will remove itself from gatheringInProgress after completion.
+     * If there are some other task for the given key - operation will be scheduled right after it if necessary
+     * (current task have never configuration or topology).
+     *
+     * @param ctx Task to register.
+     * @return {@code true} if task was actually pushed as Gathering in progress task, {@code false} - othrewise.
+     */
+    private boolean registerNewTask(LocalStatisticsGatheringContext ctx) {
+        boolean res[] = new boolean[1];
+
+        gatheringInProgress.compute(ctx.configuration().key(), (k, v) -> {
+            if (v == null) {
+                // No old context - start new
+                res[0] = true;
+
+                ctx.future().whenComplete((r, t) -> {
+                    if (t != null) {
+                        if (t instanceof CancellationException || t instanceof NodeStoppingException) {
+                            if (log.isDebugEnabled())
+                                log.debug("Got " + t.getClass() + " exception during statistics collection by key "
+                                    + ctx.configuration().key() + "." );
+                        }
+                        else
+                            log.warning("Unexpected error during statistics collection by key " +
+                                ctx.configuration().key() + " . " + t.getMessage(), t);
+                    }
+
+                    // Anyway - try to remove itself from gathering map.
+                    gatheringInProgress.remove(ctx.configuration().key(), ctx);
+                });
+
+                return ctx;
+            }
+
+            // If there are key - check if we can cancel it.
+            if (v.topologyVersion() == null ||
+                (ctx.topologyVersion() != null && v.topologyVersion().compareTo(ctx.topologyVersion()) < 0) ||
+                v.configuration().compareTo(ctx.configuration()) < 0) {
+                // Old context for older topology or config - cancel and start new
+                v.cancel();
+
+                v.future().whenComplete((r, t) -> {
+                    // Will be executed before original, so have to try to cancel previous context to add new one.
+                    gatheringInProgress.remove(ctx.configuration().key(), v);
+
+                    busyRun(() -> {
+                        updateLocalStatistics(ctx);
+                    });
+                });
+
+                res[0] = false;
+
+                return v;
+            }
+
+            res[0] = false;
+
+            return v;
+        });
+
+        return res[0];
+    }
+
+    /**
+     * Generate and subtim tasks into gathering pool for existing gathering context.
+     *
+     * @param ctx Context to generate tasks by.
+     */
+    private void submitTasks(LocalStatisticsGatheringContext ctx) {
+        for (int part : ctx.remainingParts()) {
+            final GatherPartitionStatistics task = new GatherPartitionStatistics(
+                statRepo,
+                ctx,
+                part,
+                log
+            );
+
+            submitTask(task);
+        }
+    }
+
+    /**
+     * Aggregate partition statistics to local one.
+     * @param ctx Context to use in aggregation.
+     */
+    private void aggregateStatistics(LocalStatisticsGatheringContext ctx) {
+        if (ctx.cancelled())
+            return;
+
+        StatisticsKey key = ctx.configuration().key();
+        Collection<ObjectPartitionStatisticsImpl> allParts = statRepo.getLocalPartitionsStatistics(key);
+
+        if (ctx.forceRecollect())
+            statRepo.aggregatedLocalStatistics(allParts, ctx.configuration());
+        else {
+            Set<Integer> partsToRemove = new HashSet<>();
+            Collection<ObjectPartitionStatisticsImpl> partsToAggregate = new ArrayList<>();
+
+            for (ObjectPartitionStatisticsImpl partStat : allParts) {
+                if (ctx.allParts() == null || !ctx.allParts().contains(partStat.partId()))
+                    partsToRemove.add(partStat.partId());
+                else
+                    partsToAggregate.add(partStat);
+            }
+
+            if (!partsToRemove.isEmpty())
+                statRepo.clearLocalPartitionsStatistics(ctx.configuration().key(), partsToRemove);
+
+            if (!partsToAggregate.isEmpty())
+                statRepo.aggregatedLocalStatistics(partsToAggregate, ctx.configuration());
+        }
+    }
+
+    /**
+     * Try to get busyLock and check active state. Return success flag.
+     * If unsuccess - release busyLock.
+     *
+     * @return {@code true}
+     */
+    private boolean startJob() {
+        if (!busyLock.enterBusy()) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            return false;
+        }
+
+        if (!active) {
+            if (log.isDebugEnabled())
+                log.debug("Unable to start statistics operation due to inactive state.");
+
+            busyLock.leaveBusy();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Just unlock busyLock.
+     */
+    private void endJob() {
+        busyLock.leaveBusy();
+    }
+
+    /**
+     * Mark partition task failed. If that was the last partition -
+     * finalize ctx and remove it from gathering in progress.
+     *
+     * @param ctx Context to fishish partition in.
+     * @param partId Partition id.
+     */
+    private void failPartTask(LocalStatisticsGatheringContext ctx, int partId) {
+    }
+
+    /**
+     * Submit partition gathering task.
+     *
+     * @param task Gathering task to proceed.
+     */
+    private void submitTask(final GatherPartitionStatistics task) {
+        LocalStatisticsGatheringContext ctx = task.context();
+
+        gatherPool.submit(() -> {
+            if (!startJob()) {
+                if (log.isDebugEnabled())
+                    log.debug(String.format("Gathering failed for key %s.%d ", ctx.configuration().key(),
+                        task.partition()));
+
+                ctx.partitionNotAvailable(task.partition());
+
+                return;
+            }
+
+            try {
+                task.call();
+
+                if (ctx.partitionDone(task.partition())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Local partitions statistics successfully gathered by key " +
+                            ctx.configuration().key());
+
+                    aggregateStatistics(ctx);
+
+                    ctx.future().complete(null);
+                }
+
+            }
+            catch (Throwable t) {
+                ctx.partitionNotAvailable(task.partition());
+
+                if (t instanceof GatherStatisticCancelException) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Collect statistics task was cancelled " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                    }
+                }
+                else if (t.getCause() instanceof NodeStoppingException) {
+                    if (log.isDebugEnabled())
+                        log.debug("Node stopping during statistics collection on " +
+                            "[key=" + ctx.configuration().key() + ", part=" + task.partition() + ']');
+                }
+                else
+                    log.warning("Unexpected error on statistic gathering", t);
+            }
+            finally {
+                endJob();
+            }
+
+        });
+    }
+
+    /**
+     * Check if specified object stistic fully meet specified statistics object configuration.
+     *
+     * @param stat Object statistics to test.
+     * @param cfg Statistics object configuration to compare with.
+     * @return {@code true} if specified statistics fully meet to specified configuration requiremenrs,
+     *         {@code false} - otherwise.
+     */
+    private boolean partStatSuitToConfiguration(ObjectStatisticsImpl stat, StatisticsObjectConfiguration cfg) {
+        if (stat == null)
+            return false;
+
+        if (stat.columnsStatistics().size() != cfg.columns().size())
+            return false;
+
+        for (StatisticsColumnConfiguration colCfg : cfg.columns().values()) {
+            ColumnStatistics colStat = stat.columnStatistics(colCfg.name());
+
+            if (colStat == null || colCfg.version() > colStat.version())
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Run task on busy lock.
+     *
+     * @param r Task to run.
+     */
+    public void busyRun(Runnable r) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            if (!active)
+                return;
+
+            r.run();
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+
+    }
+
+    /**
+     * Get gathering context by key.
+     *
+     * @param key Statistics key.
+     * @return Gathering in progress or {@code null} if there are no active gathering by specified key.
+     */
+    public LocalStatisticsGatheringContext gatheringInProgress(StatisticsKey key) {

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.

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

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