You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by ud...@apache.org on 2016/10/19 19:59:49 UTC

[21/50] [abbrv] incubator-geode git commit: GEODE-1981: Wrapping user ResultCollector in synchronized wrapper

GEODE-1981: Wrapping user ResultCollector in synchronized wrapper

When executing a function from a client, we can be adding results to the
result collector from multiple threads. Our docs claim the user should
not have to synchronize their result collector. One code path was already
synchronizing on the collector when adding results. However, if the
function returned an exception we were not synchronizing.

Adding a SynchronizedResultCollector and wrapping the users collector in
that to ensure that there will be no unsynchronized access of the result
collector.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/f2c3ca48
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/f2c3ca48
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/f2c3ca48

Branch: refs/heads/feature/GEODE-1874
Commit: f2c3ca489cc0826a96c30c4ffa9464cab8402b94
Parents: c2ddc96
Author: Dan Smith <up...@apache.org>
Authored: Wed Oct 12 13:54:28 2016 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Thu Oct 13 11:13:18 2016 -0700

----------------------------------------------------------------------
 .../client/internal/ExecuteFunctionOp.java      |  4 +-
 .../ExecuteRegionFunctionSingleHopOp.java       |  2 -
 .../cache/execute/ServerFunctionExecutor.java   |  5 +-
 .../execute/ServerRegionFunctionExecutor.java   |  3 +-
 .../util/SynchronizedResultCollector.java       | 57 ++++++++++++++++++++
 5 files changed, 63 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2c3ca48/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
index 6597b68..55b0fb0 100755
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteFunctionOp.java
@@ -569,9 +569,7 @@ public class ExecuteFunctionOp {
               else {
                 DistributedMember memberID = (DistributedMember)((ArrayList)resultResponse)
                     .get(1);
-                synchronized (resultCollector) {
-                  resultCollector.addResult(memberID, result);                    
-                }
+                resultCollector.addResult(memberID, result);
                 FunctionStats.getFunctionStats(this.functionId)
                     .incResultsReceived();
               }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2c3ca48/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
index 51ea8e4..f94c598 100644
--- a/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
+++ b/geode-core/src/main/java/org/apache/geode/cache/client/internal/ExecuteRegionFunctionSingleHopOp.java
@@ -396,10 +396,8 @@ public class ExecuteRegionFunctionSingleHopOp {
               else {
                 DistributedMember memberID = (DistributedMember)((ArrayList)resultResponse)
                     .get(1);
-                synchronized (this.resultCollector) {
                   this.resultCollector
                       .addResult(memberID, result);
-                }
                 FunctionStats.getFunctionStats(this.functionId,
                     this.executor.getRegion().getSystem()).incResultsReceived();
               }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2c3ca48/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerFunctionExecutor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerFunctionExecutor.java
index 4295898..1db6e86 100644
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerFunctionExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerFunctionExecutor.java
@@ -33,6 +33,7 @@ import org.apache.geode.cache.execute.FunctionException;
 import org.apache.geode.cache.execute.FunctionService;
 import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.internal.cache.TXManagerImpl;
+import org.apache.geode.internal.cache.execute.util.SynchronizedResultCollector;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 /**
  * 
@@ -73,9 +74,9 @@ public class ServerFunctionExecutor extends AbstractExecution {
     this.args = args;
   }
   
-  private ServerFunctionExecutor(ServerFunctionExecutor sfe, ResultCollector rs) {
+  private ServerFunctionExecutor(ServerFunctionExecutor sfe, ResultCollector collector) {
     this(sfe);
-    this.rc = rs;
+    this.rc = collector != null ? new SynchronizedResultCollector(collector): collector;
   }
   
   private ServerFunctionExecutor(ServerFunctionExecutor sfe, MemberMappedArgument argument) {

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2c3ca48/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerRegionFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerRegionFunctionExecutor.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerRegionFunctionExecutor.java
index b5bc684..5669ad1 100755
--- a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerRegionFunctionExecutor.java
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/ServerRegionFunctionExecutor.java
@@ -30,6 +30,7 @@ import org.apache.geode.cache.execute.ResultCollector;
 import org.apache.geode.internal.cache.GemFireCacheImpl;
 import org.apache.geode.internal.cache.LocalRegion;
 import org.apache.geode.internal.cache.TXStateProxyImpl;
+import org.apache.geode.internal.cache.execute.util.SynchronizedResultCollector;
 import org.apache.geode.internal.i18n.LocalizedStrings;
 import org.apache.geode.internal.logging.LogService;
 
@@ -103,7 +104,7 @@ public class ServerRegionFunctionExecutor extends AbstractExecution {
     this.region = serverRegionFunctionExecutor.region;
     this.filter.clear();
     this.filter.addAll(serverRegionFunctionExecutor.filter);
-    this.rc = rc;
+    this.rc = rc != null ? new SynchronizedResultCollector(rc) : null;
     this.executeOnBucketSet = serverRegionFunctionExecutor.executeOnBucketSet;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/f2c3ca48/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/SynchronizedResultCollector.java
----------------------------------------------------------------------
diff --git a/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/SynchronizedResultCollector.java b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/SynchronizedResultCollector.java
new file mode 100644
index 0000000..9e09679
--- /dev/null
+++ b/geode-core/src/main/java/org/apache/geode/internal/cache/execute/util/SynchronizedResultCollector.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.geode.internal.cache.execute.util;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.geode.cache.execute.FunctionException;
+import org.apache.geode.cache.execute.ResultCollector;
+import org.apache.geode.distributed.DistributedMember;
+
+public class SynchronizedResultCollector<T,S> implements ResultCollector<T, S> {
+
+  public final ResultCollector<T,S> collector;
+
+  public SynchronizedResultCollector(final ResultCollector collector) {
+    this.collector = collector;
+  }
+
+  @Override
+  public synchronized S getResult() throws FunctionException {
+    return collector.getResult();
+  }
+
+  @Override
+  public synchronized S getResult(final long timeout, final TimeUnit unit) throws FunctionException, InterruptedException {
+    return collector.getResult(timeout, unit);
+  }
+
+  @Override
+  public synchronized void addResult(final DistributedMember memberID, final T resultOfSingleExecution) {
+    collector.addResult(memberID, resultOfSingleExecution);
+  }
+
+  @Override
+  public synchronized void endResults() {
+    collector.endResults();
+  }
+
+  @Override
+  public synchronized void clearResults() {
+    collector.clearResults();
+  }
+}