ignite-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From yzhda...@apache.org
Subject ignite git commit: Debugging slowdowns
Date Mon, 19 Oct 2015 08:08:36 GMT
Repository: ignite
Updated Branches:
  refs/heads/ignite-1.4-slow-server-debug 9f7b524ee -> 498f83c1d


Debugging slowdowns


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/498f83c1
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/498f83c1
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/498f83c1

Branch: refs/heads/ignite-1.4-slow-server-debug
Commit: 498f83c1d5b598f92bfdae8604244bb023e2b656
Parents: 9f7b524
Author: Yakov Zhdanov <yzhdanov@gridgain.com>
Authored: Mon Oct 19 11:08:42 2015 +0300
Committer: Yakov Zhdanov <yzhdanov@gridgain.com>
Committed: Mon Oct 19 11:08:42 2015 +0300

----------------------------------------------------------------------
 .../util/future/GridCompoundFuture.java         | 90 ++++++++------------
 1 file changed, 37 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/498f83c1/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index c795578..e454b32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -33,7 +34,6 @@ import org.jetbrains.annotations.Nullable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 /**
  * Future composed of multiple inner futures.
@@ -53,10 +53,6 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     private static final AtomicIntegerFieldUpdater<GridCompoundFuture> lsnrCallsUpd
=
         AtomicIntegerFieldUpdater.newUpdater(GridCompoundFuture.class, "lsnrCalls");
 
-    /** */
-    private static final AtomicReferenceFieldUpdater<GridCompoundFuture, Throwable>
errUpd =
-        AtomicReferenceFieldUpdater.newUpdater(GridCompoundFuture.class, Throwable.class,
"err");
-
     /** Futures. */
     private final Collection<IgniteInternalFuture<T>> futs = new ArrayList<>();
 
@@ -67,20 +63,18 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     /** Exceptions to ignore. */
     private Class<? extends Throwable>[] ignoreChildFailures;
 
-    /** Error. */
-    private volatile Throwable err;
-
     /**
+     * Updated via {@link #flagsUpd}.
+     *
      * @see #INITED
      */
+    @SuppressWarnings("unused")
     private volatile int flags;
 
-    /** */
+    /** Updated via {@link #lsnrCallsUpd}. */
+    @SuppressWarnings("unused")
     private volatile int lsnrCalls;
 
-    /** */
-    private final Object mux = new Object();
-
     /**
      *
      */
@@ -128,7 +122,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
      * @return Collection of futures.
      */
     private Collection<IgniteInternalFuture<T>> futures(boolean pending) {
-        synchronized (mux) {
+        synchronized (futs) {
             Collection<IgniteInternalFuture<T>> res = new ArrayList<>(futs.size());
 
             for (IgniteInternalFuture<T> fut : futs) {
@@ -192,19 +186,20 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     public void add(IgniteInternalFuture<T> fut) {
         assert fut != null;
 
-        synchronized (mux) {
+        synchronized (futs) {
             futs.add(fut);
         }
 
         fut.listen(new Listener());
 
-        if (isCancelled())
+        if (isCancelled()) {
             try {
                 fut.cancel();
             }
             catch (IgniteCheckedException e) {
                 onDone(e);
             }
+        }
     }
 
     /**
@@ -212,7 +207,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
      *
      * @param futs Futures to add.
      */
-    public void addAll(@Nullable IgniteInternalFuture<T>... futs) {
+    @SafeVarargs
+    public final void addAll(@Nullable IgniteInternalFuture<T>... futs) {
         addAll(F.asList(futs));
     }
 
@@ -222,9 +218,10 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
      * @param futs Futures to add.
      */
     public void addAll(@Nullable Iterable<IgniteInternalFuture<T>> futs) {
-        if (futs != null)
+        if (futs != null) {
             for (IgniteInternalFuture<T> fut : futs)
                 add(fut);
+        }
     }
 
     /**
@@ -246,8 +243,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     }
 
     /**
-     * @param flag
-     * @return
+     * @param flag Flag to CAS.
+     * @return {@code True} if CAS succeeds.
      */
     private boolean casFlag(int flag) {
         for (;;) {
@@ -262,8 +259,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     }
 
     /**
-     * @param flag
-     * @return
+     * @param flag Flag to check.
+     * @return {@code True} if set.
      */
     private boolean flagSet(int flag) {
         return (flags & flag) != 0;
@@ -283,30 +280,17 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
      * Check completeness of the future.
      */
     private void checkComplete() {
-        Throwable err = this.err;
-
-        boolean ignore = ignoreFailure(err);
-
-        if (flagSet(INITED) && !isDone() &&
-            ((err != null && !ignore) || lsnrCalls == futuresSize())) {
+        if (flagSet(INITED) && !isDone() && lsnrCalls == futuresSize()) {
             try {
-                onDone(
-                    rdc != null ? rdc.reduce() : null,
-                    ignore ? null : err);
+                onDone(rdc != null ? rdc.reduce() : null);
             }
             catch (RuntimeException e) {
-                U.error(
-                    null,
-                    "Failed to execute compound future reducer: " + this,
-                    e);
+                U.error(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
             }
             catch (AssertionError e) {
-                U.error(
-                    null,
-                    "Failed to execute compound future reducer: " + this,
-                    e);
+                U.error(null, "Failed to execute compound future reducer: " + this, e);
 
                 onDone(e);
 
@@ -316,10 +300,10 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
     }
 
     /**
-     * @return
+     * @return Futures size.
      */
     private int futuresSize() {
-        synchronized (mux) {
+        synchronized (futs) {
             return futs.size();
         }
     }
@@ -349,15 +333,13 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
         return S.toString(GridCompoundFuture.class, this,
             "done", isDone(),
             "cancelled", isCancelled(),
-            "err", error()
-//            ,
-//
-// "futs",
-//                F.viewReadOnly(futs, new C1<IgniteInternalFuture<T>, String>()
{
-//                    @Override public String apply(IgniteInternalFuture<T> f) {
-//                        return Boolean.toString(f.isDone());
-//                    }
-//                })
+            "err", error(),
+            "futs",
+                F.viewReadOnly(futures(), new C1<IgniteInternalFuture<T>, String>()
{
+                    @Override public String apply(IgniteInternalFuture<T> f) {
+                        return Boolean.toString(f.isDone());
+                    }
+                })
         );
     }
 
@@ -394,18 +376,20 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R>
{
             }
             catch (IgniteTxOptimisticCheckedException | IgniteFutureCancelledCheckedException
|
                 ClusterTopologyCheckedException e) {
-                errUpd.compareAndSet(GridCompoundFuture.this, null, e);
+                if (!ignoreFailure(e))
+                    onDone(e);
             }
             catch (IgniteCheckedException e) {
-                if (!ignoreFailure(e))
+                if (!ignoreFailure(e)) {
                     U.error(null, "Failed to execute compound future reducer: " + this, e);
 
-                errUpd.compareAndSet(GridCompoundFuture.this, null, e);
+                    onDone(e);
+                }
             }
             catch (RuntimeException e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);
 
-                errUpd.compareAndSet(GridCompoundFuture.this, null, e);
+                onDone(e);
             }
             catch (AssertionError e) {
                 U.error(null, "Failed to execute compound future reducer: " + this, e);


Mime
View raw message