ivanzlenko commented on code in PR #5574:
URL: https://github.com/apache/ignite-3/pull/5574#discussion_r2030705122


##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java:
##########
@@ -503,7 +503,7 @@ private CompletableFuture<Void> handle(VersionedUpdate 
update, HybridTimestamp m
             return allOf(eventFutures.toArray(CompletableFuture[]::new))
                     .whenComplete((ignore, err) -> {
                         if (err != null) {
-                            failureManager.process(new 
FailureContext(CRITICAL_ERROR,
+                            failureProcessor.process(new 
FailureContext(CRITICAL_ERROR,

Review Comment:
   Isn't it process critical error? 



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java:
##########
@@ -280,7 +280,7 @@ private CompletableFuture<Void> initCatalog(Catalog 
emptyCatalog) {
         return updateLog.append(new VersionedUpdate(emptyCatalog.version() + 
1, 0L, entries))
                 .handle((result, error) -> {
                     if (error != null) {
-                        failureManager.process(new 
FailureContext(CRITICAL_ERROR,
+                        failureProcessor.process(new 
FailureContext(CRITICAL_ERROR,

Review Comment:
   Isn't it process critical error? 



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/ZoneRebalanceRaftGroupEventsListener.java:
##########
@@ -203,7 +209,12 @@ public void onLeaderElected(long term) {
                     }
                 } catch (Exception e) {
                     // TODO: IGNITE-14693
-                    LOG.warn("Unable to start rebalance [tablePartitionId, 
term={}]", e, zonePartitionId, term);
+                    processCriticalFailure(
+                            failureProcessor,
+                            e,
+                            "Unable to start rebalance [tablePartitionId=%s, 
term=%s]",
+                            zonePartitionId, term

Review Comment:
   ```suggestion
                               zonePartitionId,
                               term
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorage.java:
##########
@@ -290,9 +292,10 @@ private CompletableFuture<Void> 
startIndexMetaTreeDestruction(RenewablePartition
         return destroyTree(renewableState.indexMetaTree(), null)
                 .whenComplete((res, e) -> {
                     if (e != null) {
-                        LOG.error(
-                                "Index meta tree destruction failed: 
[tableId={}, partitionId={}]",
+                        processCriticalFailure(
+                                failureProcessor,
                                 e,
+                                "Index meta tree destruction failed: 
[tableId=%s, partitionId=%s]",
                                 tableStorage.getTableId(), partitionId

Review Comment:
   ```suggestion
                                   tableStorage.getTableId(), 
                                   partitionId
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/VolatilePageMemoryMvPartitionStorage.java:
##########
@@ -256,9 +257,10 @@ private CompletableFuture<Void> 
startMvDataDestruction(RenewablePartitionStorage
         return destroyTree(renewableState.versionChainTree(), chainKey -> 
destroyVersionChain((VersionChain) chainKey, renewableState))
                 .whenComplete((res, e) -> {
                     if (e != null) {
-                        LOG.error(
-                                "Version chains destruction failed: 
[tableId={}, partitionId={}]",
+                        processCriticalFailure(
+                                failureProcessor,
                                 e,
+                                "Version chains destruction failed: 
[tableId=%s, partitionId=%s]",
                                 tableStorage.getTableId(), partitionId

Review Comment:
   ```suggestion
                                   tableStorage.getTableId(), 
                                   partitionId
   ```



##########
modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/ZonePartitionReplicaListener.java:
##########
@@ -319,15 +330,16 @@ public Map<Integer, ReplicaTableProcessor> 
tableReplicaProcessors() {
     @Override
     public void onShutdown() {
         replicaProcessors.forEach((tableId, listener) -> {
-                    try {
-                        listener.onShutdown();
-                    } catch (Throwable th) {
-                        LOG.error("Error during table partition listener stop 
for [tableId="
-                                        + tableId + ", partitionId=" + 
replicationGroupId.partitionId() + "].",
-                                th
-                        );
-                    }
-                }
-        );
+            try {
+                listener.onShutdown();
+            } catch (Throwable th) {
+                processCriticalFailure(
+                        failureProcessor,
+                        th,
+                        "Error during table partition listener stop for 
[tableId=%s, partitionId=%s].",
+                        tableId, replicationGroupId.partitionId()

Review Comment:
   ```suggestion
                           tableId, 
                           replicationGroupId.partitionId()
   ```



##########
modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java:
##########
@@ -558,10 +563,12 @@ private static void logAndRethrowIfError(InNetworkObject 
obj, Throwable e) {
                 );
             }
         } else {
-            LOG.error(
-                    "onMessage() failed while processing {} from {}",
+            processCriticalFailure(
+                    failureProcessor,
                     e,
-                    LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging(), obj.sender()
+                    "onMessage() failed while processing %s from %s",
+                    LOG.isDebugEnabled() && includeSensitive() ? message : 
message.toStringForLightLogging(),

Review Comment:
   let's move into separate variable



##########
modules/core/src/main/java/org/apache/ignite/internal/failure/FailureProcessorUtils.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.failure;
+
+import static org.apache.ignite.internal.failure.FailureType.CRITICAL_ERROR;
+import static org.apache.ignite.lang.ErrorGroups.Common.INTERNAL_ERR;
+
+import org.apache.ignite.internal.lang.IgniteInternalException;
+
+/**
+ * Utils making it easier to report failures with {@link FailureProcessor}.
+ */
+public class FailureProcessorUtils {
+    /**
+     * Reports a failure to a failure processor capturing the call site 
context.
+     *
+     * @param processor Processor used to process the failure.
+     * @param th The failure.
+     * @param messageFormat Message format (same as {@link 
String#format(String, Object...)} accepts).
+     * @param args Arguments for the message.
+     */
+    public static void processCriticalFailure(FailureProcessor processor, 
Throwable th, String messageFormat, Object... args) {
+        processor.process(new FailureContext(CRITICAL_ERROR,
+                new IgniteInternalException(INTERNAL_ERR, 
String.format(messageFormat, args))

Review Comment:
   Why switching from our own formatter to String.format? 



##########
modules/network/src/main/java/org/apache/ignite/internal/network/DefaultMessagingService.java:
##########
@@ -435,11 +440,11 @@ private void handleMessageFromNetwork(InNetworkObject 
inNetworkObject) {
             try {
                 handleStartingWithFirstHandler(payload, finalCorrelationId, 
inNetworkObject, firstHandlerContext, handlerContexts);
             } catch (Throwable e) {
-                logAndRethrowIfError(inNetworkObject, e);
+                handleAndRethrowIfError(inNetworkObject, e);
             } finally {
                 long tookMillis = 
TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startedNanos);
 
-                if (tookMillis > 100) {
+                if (tookMillis > 100 && 
IgniteSystemProperties.getBoolean(IgniteSystemProperties.LONG_HANDLING_LOGGING_ENABLED,
 false)) {

Review Comment:
   Why we made this change in this PR? 
   And even if we need it I think it is better to have static import for 
LONG_HANDLING_LOGGING_ENABLED



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java:
##########
@@ -1193,11 +1199,11 @@ private CompletableFuture<?> removeDataNodesKeys(int 
zoneId, HybridTimestamp tim
                     .thenApply(StatementResult::getAsBoolean)
                     .whenComplete((invokeResult, e) -> {
                         if (e != null) {
-                            LOG.error(
-                                    "Failed to delete zone's dataNodes keys 
[zoneId = {}, timestamp = {}]",
+                            processCriticalFailure(
+                                    failureProcessor,
                                     e,
-                                    zoneId,
-                                    timestamp
+                                    "Failed to delete zone's dataNodes keys 
[zoneId = %s, timestamp = %s]",
+                                    zoneId, timestamp

Review Comment:
   ```suggestion
                                       zoneId,
                                       timestamp
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/VolatilePageMemoryTableStorage.java:
##########
@@ -176,9 +180,11 @@ CompletableFuture<Void> 
clearStorageAndUpdateDataStructures(AbstractPageMemoryMv
 
         volatilePartitionStorage.destroyStructures().whenComplete((res, ex) -> 
{
             if (ex != null) {
-                LOG.error(
-                        "Could not destroy structures: [tableId={}, 
partitionId={}]",
-                        ex, getTableId(), 
volatilePartitionStorage.partitionId()
+                processCriticalFailure(
+                        failureProcessor,
+                        ex,
+                        "Could not destroy structures: [tableId=%s, 
partitionId=%s]",
+                        getTableId(), volatilePartitionStorage.partitionId()

Review Comment:
   ```suggestion
                           getTableId(), 
                           volatilePartitionStorage.partitionId()
   ```



##########
modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java:
##########
@@ -1129,12 +1136,11 @@ CompletableFuture<?> onZoneCreate(
                     .thenApply(StatementResult::getAsBoolean)
                     .whenComplete((invokeResult, e) -> {
                         if (e != null) {
-                            LOG.error(
-                                    "Failed to initialize zone's dataNodes 
history [zoneId = {}, timestamp = {}, dataNodes = {}]",
+                            processCriticalFailure(
+                                    failureProcessor,
                                     e,
-                                    zoneId,
-                                    timestamp,
-                                    nodeNames(dataNodes)
+                                    "Failed to initialize zone's dataNodes 
history [zoneId = %s, timestamp = %s, dataNodes = %s]",
+                                    zoneId, timestamp, nodeNames(dataNodes)

Review Comment:
   ```suggestion
                                       zoneId,
                                       timestamp,
                                       nodeNames(dataNodes)
   ```



##########
modules/partition-replicator/src/main/java/org/apache/ignite/internal/partition/replicator/PartitionReplicaLifecycleManager.java:
##########
@@ -804,7 +821,11 @@ private CompletableFuture<List<Assignments>> 
writeZoneAssignmentsToMetastore(int
                                 })
                                 .whenComplete((realAssignments, e) -> {
                                     if (e != null) {
-                                        LOG.error("Couldn't get assignments 
from metastore for zone [zoneId={}].", e, zoneId);
+                                        processCriticalFailure(
+                                                failureProcessor,
+                                                e,
+                                                "Couldn't get assignments from 
metastore for zone [zoneId=%s].", zoneId

Review Comment:
   ```suggestion
                                                   "Couldn't get assignments 
from metastore for zone [zoneId=%s].",
                                                   zoneId
   ```



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1296,7 +1307,12 @@ private CompletableFuture<Void> 
startPartitionAndStartClient(
                 forcedAssignments
         ).handle((res, ex) -> {
             if (ex != null) {
-                LOG.warn("Unable to update raft groups on the node 
[tableId={}, partitionId={}]", ex, tableId, partId);
+                processCriticalFailure(
+                        failureProcessor,
+                        ex,
+                        "Unable to update raft groups on the node [tableId=%s, 
partitionId=%s]",
+                        tableId, partId

Review Comment:
   ```suggestion
                           tableId, 
                           partId
   ```



##########
modules/index/src/main/java/org/apache/ignite/internal/index/ChangeIndexStatusTask.java:
##########
@@ -189,9 +188,11 @@ CompletableFuture<Void> start() {
                                     // The index's table might have been 
dropped while we were waiting for the ability
                                     // to switch the index status to a new 
state, so IndexNotFound is not a problem.
                                     && !(cause instanceof 
IndexNotFoundValidationException)) {
-                                LOG.error("Error starting index task: {}", 
throwable, indexDescriptor.id());
-
-                                failureManager.process(new 
FailureContext(FailureType.CRITICAL_ERROR, throwable));
+                                processCriticalFailure(
+                                        failureProcessor,
+                                        throwable,
+                                        "Error starting index task: %s", 
indexDescriptor.id()

Review Comment:
   ```suggestion
                                           "Error starting index task: %s",
                                           indexDescriptor.id()
   ```



##########
modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java:
##########
@@ -1162,7 +1168,12 @@ private CompletableFuture<Void> 
startLocalPartitionsAndClients(
                             assignmentsTimestamp
                     ).whenComplete((res, ex) -> {
                         if (ex != null) {
-                            LOG.warn("Unable to update raft groups on the node 
[tableId={}, partitionId={}]", ex, tableId, partId);
+                            processCriticalFailure(
+                                    failureProcessor,
+                                    ex,
+                                    "Unable to update raft groups on the node 
[tableId=%s, partitionId=%s]",
+                                    tableId, partId

Review Comment:
   ```suggestion
                                       tableId, 
                                       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-unsubscr...@ignite.apache.org

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

Reply via email to