[jboss-cvs] JBossCache/src/org/jboss/cache/loader ...

Galder Zamarreno galder.zamarreno at jboss.com
Tue Jul 17 18:16:48 EDT 2007


  User: gzamarreno
  Date: 07/07/17 18:16:48

  Modified:    src/org/jboss/cache/loader    SingletonStoreCacheLoader.java
                        CacheLoaderManager.java
  Added:       src/org/jboss/cache/loader   
                        SingletonStoreDefaultConfig.java
  Log:
  [JBCACHE-1134] singleton store cache loader class configuration added and changed the way SSCL is configured in the XML. Also migrated push state logic to use java.util.concurrent package. Still missing, changing user guide for the updated configuration and an XML example in etc/
  
  Revision  Changes    Path
  1.6       +371 -67   JBossCache/src/org/jboss/cache/loader/SingletonStoreCacheLoader.java
  
  (In the diff below, changes in quantity of whitespace are not shown.)
  
  Index: SingletonStoreCacheLoader.java
  ===================================================================
  RCS file: /cvsroot/jboss/JBossCache/src/org/jboss/cache/loader/SingletonStoreCacheLoader.java,v
  retrieving revision 1.5
  retrieving revision 1.6
  diff -u -b -r1.5 -r1.6
  --- SingletonStoreCacheLoader.java	28 Jun 2007 18:18:04 -0000	1.5
  +++ SingletonStoreCacheLoader.java	17 Jul 2007 22:16:48 -0000	1.6
  @@ -5,6 +5,9 @@
   import org.jboss.cache.Fqn;
   import org.jboss.cache.Modification;
   import org.jboss.cache.NodeSPI;
  +import org.jboss.cache.CacheSPI;
  +import org.jboss.cache.config.CacheLoaderConfig;
  +import org.jboss.cache.config.CacheLoaderConfig.IndividualCacheLoaderConfig.SingletonStoreConfig;
   import org.jboss.cache.notifications.annotation.CacheListener;
   import org.jboss.cache.notifications.annotation.CacheStarted;
   import org.jboss.cache.notifications.annotation.CacheStopped;
  @@ -20,6 +23,14 @@
   import java.util.Map;
   import java.util.Set;
   import java.util.Vector;
  +import java.util.concurrent.ExecutionException;
  +import java.util.concurrent.ExecutorService;
  +import java.util.concurrent.Executors;
  +import java.util.concurrent.Future;
  +import java.util.concurrent.ThreadFactory;
  +import java.util.concurrent.TimeUnit;
  +import java.util.concurrent.TimeoutException;
  +import java.util.concurrent.Callable;
   
   /**
    * SingletonStoreCacheLoader is a delegating cache loader used for situations when only one node should interact with
  @@ -34,81 +45,317 @@
    * <p/>
    * Whenever the current coordinator dies (or leaves), the second in line will take over. That SingletonStoreCacheLoader
    * will then pass writes through to its underlying CacheLoader. Optionally, when a new coordinator takes over the
  - * Singleton, it can push the in-memory state to the cache cacheLoader.
  + * Singleton, it can push the in-memory state to the cache cacheLoader, within a time constraint.
    *
    * @author Bela Ban
    * @author <a href="mailto:galder.zamarreno at jboss.com">Galder Zamarreno</a>
    */
   public class SingletonStoreCacheLoader extends AbstractDelegatingCacheLoader
   {
  +   /**
  +    * Log instance.
  +    */
      private static final Log log = LogFactory.getLog(SingletonStoreCacheLoader.class);
  +
  +   /**
  +    * Name of thread that should pushing in-memory state to cache loader.
  +    */
  +   private static final String THREAD_NAME = "InMemoryToCacheLoaderPusher";
  +
  +   /**
  +    * Configuration for the SingletonStoreCacheLoader.
  +    */
  +   private SingletonStoreDefaultConfig config;
  +
  +   /**
  +    * Executor service used to submit tasks to push in-memory state.
  +    */
  +   private final ExecutorService executor;
  +
  +   /**
  +    * Future result of the in-memory push state task. This allows SingletonStoreCacheLoader to check whether there's any
  +    * push taks on going.
  +    */
  +   private Future<?> pushStateFuture; /* FutureTask guarantess a safe publication of the result */
  +
  +   /**
  +    * Address instance that allows SingletonStoreCacheLoader to find out whether it became the coordinator of the
  +    * cluster, or whether it stopped being it. This dictates whether the SingletonStoreCacheLoader is active or not.
  +    */
      private Address localAddress;
  -   private boolean active;// only active if coordinator
  -   private boolean pushStateWhenCoordinator;
  -   private Thread pushStateThread;
  -   private Object cacheListener;
   
  -   public SingletonStoreCacheLoader(CacheLoader cacheLoader, boolean pushConfiguration)
  +   /**
  +    * Whether the the current node is the coordinator and therefore SingletonStoreCacheLoader is active. Being active
  +    * means delegating calls to the underlying cache loader.  
  +    */
  +   private boolean active;
  +
  +   /**
  +    * Empty constructor so that it can instantiated using reflection.
  +    */
  +   public SingletonStoreCacheLoader()
  +   {
  +      super(null);
  +
  +      executor = Executors.newSingleThreadExecutor(new ThreadFactory()
  +      {
  +         public Thread newThread(Runnable r)
  +         {
  +            return new Thread(r, THREAD_NAME);
  +         }
  +      });
  +   }
  +
  +   /**
  +    * Sets the config for SingletonStoreCacheLoader and for the delegating cache loader.
  +    */
  +   @Override   
  +   public void setConfig(CacheLoaderConfig.IndividualCacheLoaderConfig config)
  +   {
  +      super.setConfig(config);
  +
  +      SingletonStoreConfig ssc = config.getSingletonStoreConfig();
  +      if (ssc instanceof SingletonStoreDefaultConfig)
      {
  -      super(cacheLoader);
  -      pushStateWhenCoordinator = pushConfiguration;
  -      cacheListener = new SingletonStoreListener();
  +         this.config = (SingletonStoreDefaultConfig)ssc;
  +      }
  +      else if (ssc != null)
  +      {
  +         this.config = new SingletonStoreDefaultConfig(ssc);
  +      }
  +      else
  +      {
  +         this.config = new SingletonStoreDefaultConfig();
  +      }
      }
   
  -   public Object getCacheListener()
  +   /**
  +    * Sets the cache loader's CacheSPI reference and also adds a listener to the cache that reacts to cluster topology
  +    * changes.
  +    *
  +    * @param c instance of CacheSPI
  +    */
  +   @Override
  +   public void setCache(CacheSPI c)
      {
  -      return cacheListener;
  +      super.setCache(c);
  +
  +      c.addCacheListener(new SingletonStoreListener());      
      }
   
  -   protected void activeStatusChanged(boolean newActiveState)
  +   /**
  +    * Protected constructor which should only be used from unit tests. Production code should set
  +    * pushStateWhenCoordinator using setConfig() method instead.
  +    *
  +    * @param SingletonStoreDefaultConfig configuration instance for SingletonStoreCacheLoader
  +    */
  +   protected SingletonStoreCacheLoader(SingletonStoreDefaultConfig config)
  +   {
  +      this();
  +
  +      this.config = config;
  +   }
  +
  +   /**
  +    * Returns SingletonStoreCacheLoader's configuration instance. This method has been defined for convenience reasons
  +    * when unit testing SingletonStoreCacheLoader's configuration.
  +    *
  +    * @return instance of SingletonStoreDefaultConfig
  +    */
  +   protected SingletonStoreDefaultConfig getSingletonStoreDefaultConfig()
  +   {
  +      return config;
  +   }
  +
  +   /**
  +    * Returns the Future instance of a running in-memory to cache loader push task. This method has been defined for
  +    * convenience reasons when unit testing.
  +    *
  +    * @return an instance of Future
  +    */
  +   protected Future<?> getPushStateFuture()
  +   {
  +      return pushStateFuture;
  +   }
  +
  +   /**
  +    * Method called when the node either becomes the coordinator or stops being the coordinator. If it becomes the
  +    * coordinator, it can optionally start the in-memory state transfer to the underlying cache store.
  +    *
  +    * @param newActiveState true if the node just became the coordinator, false if the nodes stopped being the coordinator.
  +    */
  +   protected void activeStatusChanged(boolean newActiveState) throws PushStateException
      {
         active = newActiveState;
         log.debug("changed mode: " + this);
  -      if (active && pushStateWhenCoordinator)
  -      {
  -         if (pushStateThread == null || !pushStateThread.isAlive())
  +      if (active && config.isPushStateWhenCoordinator())
            {
  -            pushStateThread = createPushStateThread();
  -            pushStateThread.setName("InMemoryToCacheLoaderPusher");
  -            pushStateThread.start();
  +         doPushState();
            }
  -         else
  +   }
  +
  +   /**
  +    * Factory method for the creation of a Callable task in charge of pushing in-memory state to cache loader.
  +    *
  +    * @return new instance of Callable<?> whose call() method either throws an exception or returns null if the task
  +    * was successfull.
  +    */
  +   protected Callable<?> createPushStateTask()
  +   {
  +      return new Callable()
  +      {
  +         public Object call() throws Exception
            {
  +            final boolean debugEnabled = log.isDebugEnabled();
  +
  +            if (debugEnabled) log.debug("start pushing in-memory state to cache cacheLoader");
               try
               {
  -               log.debug("joining currently running state push thread");
  -               pushStateThread.join();
  +               pushState(cache.getRoot());
  +               if (debugEnabled) log.debug("in-memory state passed to cache cacheLoader successfully");
               }
  -            catch (InterruptedException e)
  +            catch (Exception e)
               {
  -               log.error("joining existing push state thread was interrupted", e);
  +               throw e;
               }
  +
  +            return null;
            }
  +      };
         }
  +
  +   /**
  +    * Pushes the state of a specific node by reading the node's data from the cache and putting in the cache store
  +    * via the cache loader. This method is call recursively so that it iterates through the whole cache.
  +    *
  +    * @param node instance of NodeSPI to push to the cache loader
  +    * @throws Exception if there's any issues reading the data from the cache or pushing the node's data to the cache
  +    * loader.
  +    */
  +   protected void pushState(NodeSPI node) throws Exception
  +   {
  +      /* Put the node's data first */
  +      Set keys = node.getKeysDirect();
  +      Fqn fqn = node.getFqn();
  +
  +      for (Object aKey : keys)
  +      {
  +         Object value = cache.get(fqn, aKey);
  +         put(fqn, aKey, value);
      }
   
  -   protected Thread createPushStateThread()
  +      /* Navigates to the children */
  +      Collection<NodeSPI> children = node.getChildrenDirect();
  +      for (NodeSPI aChildren : children)
  +      {
  +         //Map.Entry entry = (Map.Entry) aChildren;
  +         pushState(aChildren);
  +      }
  +   }   
  +
  +   /**
  +    * Method that waits for the in-memory to cache loader state to finish. This method's called in case a push state
  +    * is already in progress and we need to wait for it to finish.
  +    *
  +    * @param future instance of Future representing the on going push task
  +    * @param timeout time to wait for the push task to finish
  +    * @param unit instance of TimeUnit representing the unit of timeout
  +    */
  +   protected void awaitForPushToFinish(Future future, int timeout, TimeUnit unit)
  +   {
  +      final boolean debugEnabled = log.isDebugEnabled();
  +      try
  +      {
  +         if (debugEnabled) log.debug("wait for state push to cache loader to finish");
  +         future.get(timeout, unit);
  +      }
  +      catch (TimeoutException e)
  +      {
  +         if (debugEnabled) log.debug("timed out waiting for state push to cache loader to finish");
  +      }
  +      catch (ExecutionException e)
  +      {
  +         if (debugEnabled) log.debug("exception reported waiting for state push to cache loader to finish");
  +      }
  +      catch (InterruptedException ie)
      {
  -      Thread t = new Thread(new Runnable()
  +         /* Re-assert the thread's interrupted status */
  +         Thread.currentThread().interrupt();
  +         if (log.isTraceEnabled()) log.trace("wait for state push to cache loader to finish was interrupted");
  +      }
  +   }
  +
  +   /**
  +    * Called when the SingletonStoreCacheLoader discovers that the node has become the coordinator and push in memory
  +    * state has been enabled. It might not actually push the state if there's an ongoing push task running, in which
  +    * case will wait for the push task to finish.
  +    *
  +    * @throws PushStateException when the push state task reports an issue.
  +    */
  +   private void doPushState() throws PushStateException
         {
  -         public void run()
  +      if (pushStateFuture == null || pushStateFuture.isDone())
            {
  -            log.debug("start pushing in-memory state to cache cacheLoader");
  +         Callable<?> task = createPushStateTask();
  +         pushStateFuture = executor.submit(task);
               try
               {
  -               pushState(cache.getRoot());
  -               log.debug("in-memory state passed to cache cacheLoader successfully");
  +            waitForTaskToFinish(pushStateFuture, config.getPushStateWhenCoordinatorTimeout(), TimeUnit.MILLISECONDS);
               }
               catch (Exception e)
               {
  -               log.error("unable to finish pushing the state", e);
  +            throw new PushStateException("unable to complete in memory state push to cache loader", e);
               }
            }
  -      });
  -      t.setDaemon(true);
  -      return t;
  +      else
  +      {
  +         /* at the most, we wait for push state timeout value. if it push task finishes earlier, this call
  +         * will stop when the push task finishes, otherwise a timeout exception will be reported */
  +         awaitForPushToFinish(pushStateFuture, config.getPushStateWhenCoordinatorTimeout(), TimeUnit.MILLISECONDS);
  +      }
  +   }
  +
  +   /**
  +    * Waits, within a time constraint, for a task to finish.
  +    *
  +    * @param future represents the task waiting to finish.
  +    * @param timeout maximum time to wait for the time to finish.
  +    * @param unit instance of TimeUnit representing the unit of timeout
  +    * @throws Exception if any issues are reported while waiting for the task to finish
  +    */
  +   private void waitForTaskToFinish(Future future, int timeout, TimeUnit unit) throws Exception
  +   {
  +      try
  +      {
  +         future.get(timeout, unit);
  +      }
  +      catch (TimeoutException e)
  +      {
  +         throw new Exception("task timed out", e);
  +      }
  +      catch (ExecutionException e)
  +      {
  +         throw e;
  +      }
  +      catch (InterruptedException e)
  +      {
  +         /* Re-assert the thread's interrupted status */
  +         Thread.currentThread().interrupt();
  +         if (log.isTraceEnabled()) log.trace("task was interrupted");
  +      }
  +      finally
  +      {
  +         /* no-op if task is completed */
  +         future.cancel(true); /* interrupt if running */
  +      }
      }
   
  +   /**
  +    * Indicates whether the current nodes is the coordinator of the cluster.
  +    *
  +    * @param newView View instance containing the new view of the cluster
  +    * @return whether the current node is the coordinator or not.
  +    */
      private boolean isCoordinator(View newView)
      {
         if (newView != null && localAddress != null)
  @@ -130,27 +377,10 @@
         return active;
      }
   
  -   private void pushState(NodeSPI node) throws Exception
  -   {
  -      /* Put the node's data first */
  -      Set keys = node.getKeysDirect();
  -      Fqn fqn = node.getFqn();
  -
  -      for (Object aKey : keys)
  -      {
  -         Object value = cache.get(fqn, aKey);
  -         put(fqn, aKey, value);
  -      }
  -
  -      /* Navigates to the children */
  -      Collection<NodeSPI> children = node.getChildrenDirect();
  -      for (NodeSPI aChildren : children)
  -      {
  -         //Map.Entry entry = (Map.Entry) aChildren;
  -         pushState(aChildren);
  -      }
  -   }
  -
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */
  +   @Override
      public Object put(Fqn name, Object key, Object value) throws Exception
      {
         if (active)
  @@ -161,6 +391,10 @@
         return null;
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */
  +   @Override
      public void put(Fqn name, Map attributes) throws Exception
      {
         if (active)
  @@ -169,6 +403,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */
  +   @Override
      public void put(List<Modification> modifications) throws Exception
      {
         if (active)
  @@ -177,6 +415,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public Object remove(Fqn fqn, Object key) throws Exception
      {
         if (active)
  @@ -187,6 +429,10 @@
         return null;
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void remove(Fqn fqn) throws Exception
      {
         if (active)
  @@ -195,6 +441,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void removeData(Fqn fqn) throws Exception
      {
         if (active)
  @@ -203,6 +453,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void prepare(Object tx, List<Modification> modifications, boolean one_phase) throws Exception
      {
         if (active)
  @@ -211,6 +465,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void commit(Object tx) throws Exception
      {
         if (active)
  @@ -219,6 +477,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void rollback(Object tx)
      {
         if (active)
  @@ -227,6 +489,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void storeEntireState(ObjectInputStream is) throws Exception
      {
         if (active)
  @@ -235,6 +501,10 @@
         }
      }
   
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public void storeState(Fqn subtree, ObjectInputStream is) throws Exception
      {
         if (active)
  @@ -243,11 +513,10 @@
         }
      }
   
  -   public Thread getPushStateThread()
  -   {
  -      return pushStateThread;
  -   }
  -
  +   /**
  +    * Calls the underlying cache loader's operation if the current node is the coordinator.
  +    */   
  +   @Override
      public String toString()
      {
         return "loc_addr=" + localAddress + ", active=" + active;
  @@ -261,6 +530,10 @@
      @CacheListener
      public class SingletonStoreListener
      {
  +      /**
  +       * Cache started, check whether the node is the coordinator and set the singleton store cache loader's active
  +       * status.
  +       */
         @CacheStarted
         public void cacheStarted(Event e)
         {
  @@ -275,6 +548,11 @@
            if (log.isDebugEnabled()) log.debug("cache stopped: " + this);
         }
   
  +      /**
  +       * The cluster formation changed, so determine whether the current node stopped being the coordinator or became
  +       * the coordinator. This method can lead to an optional in memory to cache loader state push, if the current node
  +       * became the coordinator. This method will report any issues that could potentially arise from this push.
  +       */
         @ViewChanged
         public void viewChange(ViewChangedEvent event)
         {
  @@ -282,8 +560,34 @@
   
            if (active != tmp)
            {
  +            try
  +            {
               activeStatusChanged(tmp);
            }
  +            catch (PushStateException e)
  +            {
  +               log.error("exception reported changing nodes active status", e);
  +            }
  +
  +         }
  +      }
  +   }
  +
  +   /**
  +    * Exception representing any issues that arise from pushing the in-memory state to the cache loader. 
  +    */
  +   public static class PushStateException extends Exception
  +   {
  +      private static final long serialVersionUID = 5542893943730200886L;
  +
  +      public PushStateException(String message, Throwable cause)
  +      {
  +         super(message, cause);
  +      }
  +
  +      public PushStateException(Throwable cause)
  +      {
  +         super(cause);
         }
      }
   }
  \ No newline at end of file
  
  
  
  1.35      +33 -8     JBossCache/src/org/jboss/cache/loader/CacheLoaderManager.java
  
  (In the diff below, changes in quantity of whitespace are not shown.)
  
  Index: CacheLoaderManager.java
  ===================================================================
  RCS file: /cvsroot/jboss/JBossCache/src/org/jboss/cache/loader/CacheLoaderManager.java,v
  retrieving revision 1.34
  retrieving revision 1.35
  diff -u -b -r1.34 -r1.35
  --- CacheLoaderManager.java	17 Jul 2007 17:39:36 -0000	1.34
  +++ CacheLoaderManager.java	17 Jul 2007 22:16:48 -0000	1.35
  @@ -14,6 +14,7 @@
   import org.jboss.cache.Fqn;
   import org.jboss.cache.config.CacheLoaderConfig;
   import org.jboss.cache.config.CacheLoaderConfig.IndividualCacheLoaderConfig;
  +import org.jboss.cache.config.CacheLoaderConfig.IndividualCacheLoaderConfig.SingletonStoreConfig;
   
   import java.util.ArrayList;
   import java.util.Iterator;
  @@ -54,6 +55,7 @@
    * </code>
    *
    * @author <a href="mailto:manik at jboss.org">Manik Surtani (manik at jboss.org)</a>
  + * @author <a href="mailto:galder.zamarreno at jboss.com">Galder Zamarreno</a>
    */
   public class CacheLoaderManager
   {
  @@ -123,7 +125,8 @@
               {
                  throw new Exception("Invalid cache loader configuration!!  Only ONE cache loader may have fetchPersistentState set to true.  Cache will not start!");
               }
  -            if (cfg.isSingletonStore() && config.isShared())
  +            SingletonStoreConfig ssc = cfg.getSingletonStoreConfig();
  +            if (ssc != null && ssc.isSingletonStoreEnabled() && config.isShared())
               {
                  throw new Exception("Invalid cache loader configuration!!  If a cache loader is configured as a singleton, the cache loader cannot be shared in a cluster!");
               }
  @@ -175,17 +178,31 @@
            }
   
            // singleton?
  -         if (cfg.isSingletonStore())
  +         SingletonStoreConfig ssc = cfg.getSingletonStoreConfig();
  +         if (ssc != null && ssc.isSingletonStoreEnabled())
            {
  -            SingletonStoreCacheLoader singletonDecorator = new SingletonStoreCacheLoader(tmpLoader, cfg.isPushStateWhenCoordinator());
  -            addCacheListener(cache, singletonDecorator.getCacheListener());
  +            Object decorator = createInstance(ssc.getSingletonStoreClass());
  +
  +            /* class providing singleton store functionality must extend AbstractDelegatingCacheLoader so that
  +            * underlying cacheloader can be set. */
  +            if (decorator instanceof AbstractDelegatingCacheLoader)
  +            {
  +               AbstractDelegatingCacheLoader singletonDecorator = (AbstractDelegatingCacheLoader)decorator;
  +               /* set the cache loader to where calls will be delegated by the class providing the singleton
  +               * store functionality. */
  +               singletonDecorator.setCacheLoader(tmpLoader);
               tmpLoader = singletonDecorator;
            }
  +            else
  +            {
  +               throw new Exception("Invalid cache loader configuration!! Singleton store implementation class must extend org.jboss.cache.loader.AbstractDelegatingCacheLoader");
  +            }
  +         }
   
            // load props
            tmpLoader.setConfig(cfg);
   
  -         tmpLoader.setCache(cache);
  +         setCacheInLoader(cache, tmpLoader);
            // we should not be creating/starting the cache loader here - this should be done in the separate
            // startCacheLoader() method.
            //           tmpLoader.create();
  @@ -198,13 +215,21 @@
         return tmpLoader;
      }
   
  -   protected void addCacheListener(Cache c, Object listener)
  +   /**
  +    * Sets the cache instance associated with the given cache loader. This method was created for testing purpouses
  +    * so that it can be overriden in the mock version of the CacheLoaderManager.
  +    *
  +    * @param c instance of cache to be set in cache loader
  +    * @param loader cache loader to which assign the cache instance
  +    */
  +   protected void setCacheInLoader(CacheSPI c, CacheLoader loader)
      {
  -      c.addCacheListener(listener);
  +      loader.setCache(c);      
      }
   
      private CacheLoader createInstance(String className) throws ClassNotFoundException, IllegalAccessException, InstantiationException
      {
  +      if (log.isTraceEnabled()) log.trace("instantiating class " + className);
         Class cl = Thread.currentThread().getContextClassLoader().loadClass(className);
         return (CacheLoader) cl.newInstance();
      }
  
  
  
  1.1      date: 2007/07/17 22:16:48;  author: gzamarreno;  state: Exp;JBossCache/src/org/jboss/cache/loader/SingletonStoreDefaultConfig.java
  
  Index: SingletonStoreDefaultConfig.java
  ===================================================================
  package org.jboss.cache.loader;
  
  import org.jboss.cache.config.CacheLoaderConfig.IndividualCacheLoaderConfig.SingletonStoreConfig;
  import org.jboss.cache.config.Dynamic;
  
  import java.util.Properties;
  
  /**
   * Default singleton store cache loader configuration implementation, which is provided with the default singleton store
   * cache loader implementation. It provides with the capability of defining whether to push the in memory state to cache
   * loader when becoming the coordinator within a time constraint. 
   *
   * @author <a href="mailto:galder.zamarreno at jboss.com">Galder Zamarreno</a>
   */
  public class SingletonStoreDefaultConfig extends SingletonStoreConfig
  {
     private static final long serialVersionUID = -5828927920142613537L;
  
     /**
      * Boolean indicating whether push state when coordinator has been configured.
      */
     @Dynamic
     private boolean pushStateWhenCoordinator;
  
     /**
      * Number of milliseconds configured defining the time constraint for the state push.
      */
     @Dynamic
     private int pushStateWhenCoordinatorTimeout;
  
     /**
      * Default constructor that sets default values for singleton store cache loader configuration taking in account
      * that this configuration belongs to the default singleton store cache loader implementation. 
      */   
     public SingletonStoreDefaultConfig()
     {
        /* pushStateWhenCoordinator enabled by default with 20 seconds as default timeout*/
        pushStateWhenCoordinator = true;
        pushStateWhenCoordinatorTimeout = 20000;
  
        /* if we got to this point, we know that singleton store must have been enabled */
        setSingletonStoreEnabled(true);
        /* and we also know that the configuration was created by SingletonStoreCacheLoader */
        setSingletonStoreClass(SingletonStoreCacheLoader.class.getName());
     }
  
     /**
      * Constructor that sets the assumed values for the default singleton store cache loader implementation and also
      * the properties, as per the properties section defined in the XML configuration.
      *
      * @param base contains properties set in XML configuration
      */
     public SingletonStoreDefaultConfig(SingletonStoreConfig base)
     {
        this();
        setSingletonStoreproperties(base.getSingletonStoreproperties());
     }
  
     @Override
     public boolean isSingletonStoreEnabled()
     {
        return true;
     }
  
     @Override
     public void setSingletonStoreEnabled(boolean singletonStoreEnabled)
     {
        /* ignore it */      
     }
  
     @Override
     public String getSingletonStoreClass()
     {
        return SingletonStoreCacheLoader.class.getName();
     }
  
     @Override
     public void setSingletonStoreClass(String singletonStoreClass)
     {
        /* ignore it */
     }
  
     /**
      * Takes the properties defined and populates the individual instance fields of the default singleton store cache
      * loader configuration.
      * 
      * @param props is an instance of Properties containing these values.
      */
     @Override
     public void setSingletonStoreproperties(Properties props)
     {
        super.setSingletonStoreproperties(props);
        String pushStateWhenCoordinatorStr = props.getProperty("pushStateWhenCoordinator");
        if (pushStateWhenCoordinatorStr != null)
        {
           /* if not null, we use the defined value, otherwise we leave it to the default value, true */
           /* note: default value for a null property is false, hence the check */
           setPushStateWhenCoordinator(Boolean.valueOf(pushStateWhenCoordinatorStr));
        }
        String pushStateWhenCoordinatorTimeoutStr = props.getProperty("pushStateWhenCoordinatorTimeout");
        if (pushStateWhenCoordinatorTimeoutStr != null)
        {
           setPushStateWhenCoordinatorTimeout(Integer.parseInt(pushStateWhenCoordinatorTimeoutStr));
        }
     }
  
     public boolean isPushStateWhenCoordinator()
     {
        return pushStateWhenCoordinator;
     }
  
     public void setPushStateWhenCoordinator(boolean pushStateWhenCoordinator)
     {
        testImmutability("pushStateWhenCoordinator");
        this.pushStateWhenCoordinator = pushStateWhenCoordinator;
     }
  
     public int getPushStateWhenCoordinatorTimeout()
     {
        return pushStateWhenCoordinatorTimeout;
     }
  
     public void setPushStateWhenCoordinatorTimeout(int pushStateWhenCoordinatorTimeout)
     {
        testImmutability("pushStateWhenCoordinatorTimeout");
        this.pushStateWhenCoordinatorTimeout = pushStateWhenCoordinatorTimeout;
     }
  
     @Override
     public boolean equals(Object obj)
     {
        if (this == obj)
           return true;
  
        if (obj instanceof SingletonStoreDefaultConfig)
        {
           SingletonStoreDefaultConfig other = (SingletonStoreDefaultConfig) obj;
           return (other.pushStateWhenCoordinator == this.pushStateWhenCoordinator)
                   && (other.pushStateWhenCoordinatorTimeout == this.pushStateWhenCoordinatorTimeout);
        }
        return false;
     }
  
  
     @Override
     public int hashCode()
     {
        int result = 13;
        result = 23 * result + (pushStateWhenCoordinator ? 0 : 1);
        result = 23 * result + pushStateWhenCoordinatorTimeout;
        return result;
     }
  }
  
  
  



More information about the jboss-cvs-commits mailing list