You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by up...@apache.org on 2015/11/17 19:58:18 UTC

incubator-geode git commit: GEODE-192: Removing TransactionFunctionService

Repository: incubator-geode
Updated Branches:
  refs/heads/develop e8ddd3398 -> 3fc299230


GEODE-192: Removing TransactionFunctionService

This internal class was added for a use case that is no longer
supported. It has a bug, in that the client may never have connected to
the server that it supposed to be able to look up and invoke the
function on. Deleting the class rather than trying to fix the bug
because the class is no longer needed.


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

Branch: refs/heads/develop
Commit: 3fc29923007f92011f4bfd4e985d1854536a98cb
Parents: e8ddd33
Author: Dan Smith <up...@apache.org>
Authored: Tue Oct 27 13:35:49 2015 -0700
Committer: Dan Smith <up...@apache.org>
Committed: Tue Nov 17 10:53:56 2015 -0800

----------------------------------------------------------------------
 .../client/internal/EndpointManagerImpl.java    |   2 -
 .../cache/execute/ServerFunctionExecutor.java   |  18 --
 .../execute/TransactionFunctionService.java     | 193 -------------------
 .../cache/execute/util/CommitFunction.java      |   1 -
 .../cache/execute/util/RollbackFunction.java    |   1 -
 .../cache/ClientServerTransactionDUnitTest.java |  31 +--
 6 files changed, 8 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
index 5c03be8..853ab36 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/cache/client/internal/EndpointManagerImpl.java
@@ -34,7 +34,6 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.ServerLocation;
 import com.gemstone.gemfire.internal.DummyStatisticsFactory;
 import com.gemstone.gemfire.internal.cache.PoolStats;
-import com.gemstone.gemfire.internal.cache.execute.TransactionFunctionService;
 import com.gemstone.gemfire.internal.cache.tier.InternalClientMembership;
 import com.gemstone.gemfire.internal.logging.LogService;
 
@@ -59,7 +58,6 @@ public class EndpointManagerImpl implements EndpointManager {
     this.cancelCriterion = cancelCriterion;
     this.poolStats = poolStats;
     listener.addListener(new EndpointListenerForBridgeMembership());
-    listener.addListener(new TransactionFunctionService.ListenerForTransactionFunctionService());
   }
   
   /* (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
index ba87bb5..bf00268 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/ServerFunctionExecutor.java
@@ -45,8 +45,6 @@ public class ServerFunctionExecutor extends AbstractExecution {
 
   private final boolean allServers;
   
-  private boolean onBehalfOfTXFunctionService;
-
   private String[] groups;
 
   public ServerFunctionExecutor(Pool p, boolean allServers, String... groups) {
@@ -68,7 +66,6 @@ public class ServerFunctionExecutor extends AbstractExecution {
       this.pool = sfe.pool;
     }
     this.allServers = sfe.allServers;
-    this.onBehalfOfTXFunctionService = sfe.onBehalfOfTXFunctionService;
     this.groups = sfe.groups;
   }
   
@@ -114,9 +111,6 @@ public class ServerFunctionExecutor extends AbstractExecution {
     }
     } finally {
       UserAttributes.userAttributes.set(null);
-      if (isOnBehalfOfTXFunctionService()) {
-        pool.releaseServerAffinity();
-      }
     }
   }
 
@@ -148,9 +142,6 @@ public class ServerFunctionExecutor extends AbstractExecution {
     }
     finally {
       UserAttributes.userAttributes.set(null);
-      if (isOnBehalfOfTXFunctionService()) {
-        pool.releaseServerAffinity();
-      }
     }
 
   }
@@ -434,13 +425,4 @@ public class ServerFunctionExecutor extends AbstractExecution {
       return executeFunction(functionObject);      
     }
   }
-
-  public void setOnBehalfOfTXFunctionService(boolean onBehalfOfTXFunctionService) {
-    this.onBehalfOfTXFunctionService = onBehalfOfTXFunctionService;
-  }
-
-  public boolean isOnBehalfOfTXFunctionService() {
-    return onBehalfOfTXFunctionService;
-  }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/TransactionFunctionService.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/TransactionFunctionService.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/TransactionFunctionService.java
deleted file mode 100644
index e56e647..0000000
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/TransactionFunctionService.java
+++ /dev/null
@@ -1,193 +0,0 @@
-/*
- * 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 com.gemstone.gemfire.internal.cache.execute;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import com.gemstone.gemfire.cache.TransactionDataNodeHasDepartedException;
-import com.gemstone.gemfire.cache.TransactionId;
-import com.gemstone.gemfire.cache.client.Pool;
-import com.gemstone.gemfire.cache.client.internal.Endpoint;
-import com.gemstone.gemfire.cache.client.internal.EndpointManager;
-import com.gemstone.gemfire.cache.client.internal.PoolImpl;
-import com.gemstone.gemfire.cache.execute.Execution;
-import com.gemstone.gemfire.cache.execute.FunctionService;
-import com.gemstone.gemfire.distributed.DistributedMember;
-import com.gemstone.gemfire.distributed.internal.ServerLocation;
-import com.gemstone.gemfire.internal.cache.GemFireCacheImpl;
-import com.gemstone.gemfire.internal.cache.PoolManagerImpl;
-import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.cache.execute.util.CommitFunction;
-import com.gemstone.gemfire.internal.cache.execute.util.RollbackFunction;
-
-/**
- * Access point for executing functions that should be targeted to
- * execute on member where the transaction is hosted.
- * </p> To commit a transaction which is not hosted locally, this
- * functionService can be used to obtain an execution object on which
- * {@link CommitFunction} and {@link RollbackFunction} can be called.
- * 
- * @author sbawaska
- * @since 6.6.1
- */
-public final class TransactionFunctionService {
-
-  private TransactionFunctionService() {
-  }
-
-  /**
-   * This is a convenience method to obtain execution objects for transactional
-   * functions. This method can be invoked from clients as well as peer members.
-   * When invoked from a client this method makes sure that the function is
-   * executed on the member where the transaction is hosted. This prevents the
-   * need for fanning out the function to multiple servers. e.g. A remotely
-   * hosted transaction can be committed as:
-   * 
-   * <pre>
-   * Execution exe = TransactionFunctionService.onTransaction(txId);
-   * List l = (List) exe.execute(transactionalFunction).getResult();
-   * </pre>
-   * 
-   * @param transactionId
-   *          the transaction identifier
-   * @return the execution object
-   * @see CommitFunction
-   * @see RollbackFunction
-   * @since 6.6.1
-   */
-  public static Execution onTransaction(TransactionId transactionId) {
-    Execution execution = null;
-    GemFireCacheImpl cache = GemFireCacheImpl
-        .getExisting("getting transaction execution");
-    if (cache.hasPool()) {
-      if (cache.getLoggerI18n().fineEnabled()) {
-        cache.getLoggerI18n().fine("TxFunctionService: inClientMode");
-      }
-      PoolImpl pool = (PoolImpl) getPool(cache);
-      
-      DistributedMember member = ((TXId)transactionId).getMemberId();
-      ServerLocation loc = memberToServerLocation.get(member);
-      if (loc == null) {
-        loc = getServerLocationForMember(pool, member);
-        if (loc == null) {
-          throw new TransactionDataNodeHasDepartedException("Could not connect to member:"+member);
-        }
-        memberToServerLocation.put(member, loc);
-      }
-      // setup server affinity so that the function is executed
-      // on the member hosting the transaction
-      pool.setupServerAffinity(false/*allowFailover*/);
-      pool.setServerAffinityLocation(loc);
-      if (cache.getLoggerI18n().fineEnabled()) {
-        cache.getLoggerI18n().fine(
-            "TxFunctionService: setting server affinity to:" + loc
-                + " which represents member:" + member);
-      }
-      execution = FunctionService.onServer(pool).withArgs(
-          transactionId);
-      // this serverAffinity is cleared up in the finally blocks
-      // of ServerFunctionExecutor
-      ((ServerFunctionExecutor)execution).setOnBehalfOfTXFunctionService(true);
-    } else {
-      if (cache.getLoggerI18n().fineEnabled()) {
-        cache.getLoggerI18n().fine("TxFunctionService: inPeerMode");
-      }
-      TXId txId = (TXId) transactionId;
-      DistributedMember member = txId.getMemberId();
-      execution = FunctionService
-          .onMember(cache.getDistributedSystem(), member).withArgs(
-              transactionId);
-    }
-    return execution;
-  }
-  
-  /**
-   * keep a mapping of DistributedMemeber to ServerLocation so that we can still
-   * connect to the same server when the poolLoadConditioningMonitor closes
-   * connection. fix for bug 43810
-   */
-  private static final ConcurrentMap<DistributedMember, ServerLocation> memberToServerLocation =
-    new ConcurrentHashMap<DistributedMember, ServerLocation>();
-  
-  /**
-   * Get the ServerLocation corresponding to the given DistributedMember
-   * or null if there is no endPoint to the member
-   * @param pool
-   * @param member
-   */
-  private static ServerLocation getServerLocationForMember(PoolImpl pool,
-      DistributedMember member) {
-    Map<ServerLocation, Endpoint> endPoints = pool.getEndpointMap();
-    for (Endpoint endPoint : endPoints.values()) {
-      if (endPoint.getMemberId().equals(member)) {
-        return endPoint.getLocation();
-      }
-    }
-    return null;
-  }
-
-  private static Pool getPool(GemFireCacheImpl cache) {
-    Pool pool = cache.getDefaultPool();
-    if (pool == null) {
-      Collection<Pool> pools = getPools();
-      if (pools.size() > 1) {
-        throw new IllegalStateException("More than one pools found");
-      }
-      pool = pools.iterator().next();
-    }
-    return pool;
-  }
-  
-  private static Collection<Pool> getPools() {
-    Collection<Pool> pools = PoolManagerImpl.getPMI().getMap().values();
-    for(Iterator<Pool> itr= pools.iterator(); itr.hasNext(); ) {
-      PoolImpl pool = (PoolImpl) itr.next();
-      if(pool.isUsedByGateway()) {
-        itr.remove();
-      }
-    }
-    return pools;
-  }
-  
-  /**
-   * When an EndPoint becomes available we remember the mapping of
-   * DistributedMember to ServerLocation so that we can reconnect to the server.
-   * fix for bug 43817
-   * 
-   * @author sbawaska
-   */
-  public static class ListenerForTransactionFunctionService implements
-      EndpointManager.EndpointListener {
-
-    public void endpointNoLongerInUse(Endpoint endpoint) {
-    }
-
-    public void endpointCrashed(Endpoint endpoint) {
-    }
-
-    public void endpointNowInUse(Endpoint endpoint) {
-      assert endpoint != null;
-      memberToServerLocation.put(endpoint.getMemberId(),
-          endpoint.getLocation());
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
index bafafa9..b0f4d93 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/CommitFunction.java
@@ -34,7 +34,6 @@ import com.gemstone.gemfire.cache.execute.FunctionInvocationTargetException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.cache.execute.TransactionFunctionService;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
index 7dbfc62..459cd47 100644
--- a/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
+++ b/gemfire-core/src/main/java/com/gemstone/gemfire/internal/cache/execute/util/RollbackFunction.java
@@ -33,7 +33,6 @@ import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.distributed.DistributedMember;
 import com.gemstone.gemfire.internal.cache.TXId;
-import com.gemstone.gemfire.internal.cache.execute.TransactionFunctionService;
 import com.gemstone.gemfire.internal.logging.LogService;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/3fc29923/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
----------------------------------------------------------------------
diff --git a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
index b361110..b5e8437 100644
--- a/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
+++ b/gemfire-core/src/test/java/com/gemstone/gemfire/internal/cache/ClientServerTransactionDUnitTest.java
@@ -17,6 +17,7 @@
 package com.gemstone.gemfire.internal.cache;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -39,6 +40,7 @@ import com.gemstone.gemfire.cache.client.ClientCache;
 import com.gemstone.gemfire.cache.client.ClientCacheFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionFactory;
 import com.gemstone.gemfire.cache.client.ClientRegionShortcut;
+import com.gemstone.gemfire.cache.client.Pool;
 import com.gemstone.gemfire.cache.client.PoolFactory;
 import com.gemstone.gemfire.cache.client.PoolManager;
 import com.gemstone.gemfire.cache.execute.Execution;
@@ -56,7 +58,6 @@ import com.gemstone.gemfire.distributed.DistributedSystem;
 import com.gemstone.gemfire.distributed.internal.InternalDistributedSystem;
 import com.gemstone.gemfire.distributed.internal.membership.InternalDistributedMember;
 import com.gemstone.gemfire.internal.AvailablePort;
-import com.gemstone.gemfire.internal.cache.execute.TransactionFunctionService;
 import com.gemstone.gemfire.internal.cache.execute.data.CustId;
 import com.gemstone.gemfire.internal.cache.execute.data.Customer;
 import com.gemstone.gemfire.internal.cache.execute.data.Order;
@@ -2607,22 +2608,6 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
     }
   }
   
-  class TestCommitFunction extends FunctionAdapter {
-    @Override
-    public void execute(FunctionContext context) {
-      CacheTransactionManager mgr = getCache().getCacheTransactionManager();
-      TransactionId txId = (TransactionId) context.getArguments();
-      assertTrue(mgr.isSuspended(txId));
-      mgr.resume(txId);
-      mgr.commit();
-      context.getResultSender().lastResult(Boolean.TRUE);
-    }
-    @Override
-    public String getId() {
-      return "CommitFunction";
-    }
-  }
-  
   public void testBasicResumableTX() {
     disconnectAllFromDS();
     Host host = Host.getHost(0);
@@ -2649,7 +2634,7 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
         result = (List) FunctionService.onRegion(cust).withArgs(args).execute(new TXFunction()).getResult();
         TransactionId txId2 = (TransactionId) result.get(0);
         assertEquals(txId, txId2);
-        result = (List) TransactionFunctionService.onTransaction(txId).execute(new TestCommitFunction()).getResult();
+        result = (List) FunctionService.onServer(getCache()).withArgs(txId).execute(new CommitFunction()).getResult();
         Boolean b = (Boolean) result.get(0);
         assertEquals(Boolean.TRUE, b);
         assertEquals(new Customer("name0", "address0"), cust.get(new CustId(0)));
@@ -2667,8 +2652,8 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
   public void testClientCommitFunction() {
     doFunctionWork(true);
   }
-  @Ignore("Bug 52331")
-  public void DISABLED_testClientRollbackFunction() {
+  
+  public void testClientRollbackFunction() {
     doFunctionWork(false);
   }
   private void doFunctionWork(final boolean commit) {
@@ -2741,9 +2726,9 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
           }
           List list = null;
           if (commit) {
-            list = (List) TransactionFunctionService.onTransaction(txId).execute(new CommitFunction()).getResult();
+            list = (List) FunctionService.onServer(getCache()).withArgs(txId).execute(new CommitFunction()).getResult();
           } else {
-            list = (List) TransactionFunctionService.onTransaction(txId).execute(new RollbackFunction()).getResult();
+            list = (List) FunctionService.onServer(getCache()).withArgs(txId).execute(new RollbackFunction()).getResult();
           }
           assertEquals(Boolean.TRUE, list.get(0));
           if (commit) {
@@ -2889,7 +2874,7 @@ public void testClientCommitAndDataStoreGetsEvent() throws Exception {
     
     accessor.invoke(new SerializableCallable() {
       public Object call() throws Exception {
-        Execution exe = TransactionFunctionService.onTransaction(txId);
+        Execution exe = FunctionService.onMember(((TXId) txId).getMemberId()).withArgs(txId);
         List list = null;
         if (commit) {
           list = (List) exe.execute(new CommitFunction()).getResult();