You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tapestry.apache.org by hl...@apache.org on 2010/07/12 20:47:41 UTC

svn commit: r963429 - in /tapestry/tapestry5/trunk/tapestry-core/src: main/java/org/apache/tapestry5/ main/java/org/apache/tapestry5/internal/services/ main/java/org/apache/tapestry5/internal/structure/ main/java/org/apache/tapestry5/services/ test/res...

Author: hlship
Date: Mon Jul 12 18:47:41 2010
New Revision: 963429

URL: http://svn.apache.org/viewvc?rev=963429&view=rev
Log:
TAP5-1197: Implement a non-pooling RequestPageCache and related code, make non-pooling the default

Added:
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java   (with props)
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java   (with props)
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java   (with props)
Modified:
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/SymbolConstants.java
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/InternalModule.java
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCache.java
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCacheImpl.java
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/structure/PageImpl.java
    tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/services/TapestryModule.java
    tapestry/tapestry5/trunk/tapestry-core/src/test/resources/org/apache/tapestry5/integration/reload/web.xml

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/SymbolConstants.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/SymbolConstants.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/SymbolConstants.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/SymbolConstants.java Mon Jul 12 18:47:41 2010
@@ -15,6 +15,7 @@
 package org.apache.tapestry5;
 
 import org.apache.tapestry5.internal.services.AssetDispatcher;
+import org.apache.tapestry5.services.ComponentClassTransformWorker;
 import org.apache.tapestry5.services.assets.AssetPathConstructor;
 import org.apache.tapestry5.services.javascript.JavascriptStack;
 
@@ -312,4 +313,14 @@ public class SymbolConstants
      */
     public static final String COMPACT_JSON = "tapestry.compact-json";
 
+    /**
+     * If "true", then Tapestry 5.1 (and earlier) style page pooling will be used. The default is "false", to
+     * allow full use of page singleton. Enabling page pooling is only necessary if an application (or library)
+     * has created {@linkplain ComponentClassTransformWorker class transformations} that introduce new, mutable
+     * fields into component classes. That's a very rare thing (most created fields contain immutable data).
+     * 
+     * @deprecated To be removed, along with the remnants of page pooling, in Tapestry 5.3.
+     * @since 5.2.0
+     */
+    public static final String PAGE_POOL_ENABLED = "tapestry.page-pool-enabled";
 }

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/InternalModule.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/InternalModule.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/InternalModule.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/InternalModule.java Mon Jul 12 18:47:41 2010
@@ -26,6 +26,7 @@ import org.apache.tapestry5.ioc.ObjectLo
 import org.apache.tapestry5.ioc.OrderedConfiguration;
 import org.apache.tapestry5.ioc.ScopeConstants;
 import org.apache.tapestry5.ioc.ServiceBinder;
+import org.apache.tapestry5.ioc.ServiceResources;
 import org.apache.tapestry5.ioc.annotations.Autobuild;
 import org.apache.tapestry5.ioc.annotations.Marker;
 import org.apache.tapestry5.ioc.annotations.Primary;
@@ -110,9 +111,24 @@ public class InternalModule
     }
 
     @Scope(ScopeConstants.PERTHREAD)
-    public static RequestPageCache buildRequestPageCache(@Autobuild
-    RequestPageCacheImpl service, PerthreadManager perthreadManager)
+    public static RequestPageCache buildRequestPageCache(ServiceResources serviceResources,
+            @Symbol(SymbolConstants.PAGE_POOL_ENABLED)
+            boolean pagePoolEnabled, PerthreadManager perthreadManager)
     {
+        if (pagePoolEnabled)
+        {
+
+            RequestPageCacheImpl service = serviceResources.autobuild(RequestPageCacheImpl.class);
+
+            perthreadManager.addThreadCleanupListener(service);
+
+            return service;
+        }
+
+        // Modern, non-pooling
+
+        NonPoolingRequestPageCacheImpl service = serviceResources.autobuild(NonPoolingRequestPageCacheImpl.class);
+
         perthreadManager.addThreadCleanupListener(service);
 
         return service;
@@ -129,8 +145,8 @@ public class InternalModule
 
     ClasspathURLConverter classpathURLConverter)
     {
-        ComponentInstantiatorSourceImpl source = new ComponentInstantiatorSourceImpl(logger, classFactory
-                .getClassLoader(), transformer, internalRequestGlobals, classpathURLConverter);
+        ComponentInstantiatorSourceImpl source = new ComponentInstantiatorSourceImpl(logger,
+                classFactory.getClassLoader(), transformer, internalRequestGlobals, classpathURLConverter);
 
         updateListenerHub.addUpdateListener(source);
 
@@ -192,6 +208,30 @@ public class InternalModule
         return service;
     }
 
+    public PageSource buildPageSource(@Autobuild
+    PageSourceImpl service,
+
+    @ComponentTemplates
+    InvalidationEventHub templatesHub,
+
+    @ComponentMessages
+    InvalidationEventHub messagesHub)
+    {
+        // This covers invalidations due to changes to classes
+
+        classesInvalidationEventHub.addInvalidationListener(service);
+
+        // This covers invalidation due to changes to message catalogs (properties files)
+
+        messagesHub.addInvalidationListener(service);
+
+        // ... and this covers invalidations due to changes to templates
+
+        templatesHub.addInvalidationListener(service);
+
+        return service;
+    }
+
     public ComponentClassCache buildComponentClassCache(@Autobuild
     ComponentClassCacheImpl service)
     {

Added: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java?rev=963429&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java (added)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java Mon Jul 12 18:47:41 2010
@@ -0,0 +1,91 @@
+// Copyright 2010 The Apache Software Foundation
+//
+// Licensed 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.tapestry5.internal.services;
+
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.tapestry5.internal.structure.Page;
+import org.apache.tapestry5.ioc.internal.util.CollectionFactory;
+import org.apache.tapestry5.ioc.internal.util.InternalUtils;
+import org.apache.tapestry5.ioc.services.ThreadCleanupListener;
+import org.apache.tapestry5.ioc.services.ThreadLocale;
+import org.apache.tapestry5.services.ComponentClassResolver;
+import org.slf4j.Logger;
+
+public class NonPoolingRequestPageCacheImpl implements RequestPageCache, ThreadCleanupListener
+{
+    private final Logger logger;
+
+    private final ComponentClassResolver resolver;
+
+    private final PageSource pageSource;
+
+    private final ThreadLocale threadLocale;
+
+    private final Map<String, Page> cache = CollectionFactory.newMap();
+
+    public NonPoolingRequestPageCacheImpl(Logger logger, ComponentClassResolver resolver, PageSource pageSource,
+            ThreadLocale threadLocale)
+    {
+        this.logger = logger;
+        this.resolver = resolver;
+        this.pageSource = pageSource;
+        this.threadLocale = threadLocale;
+    }
+
+    public void threadDidCleanup()
+    {
+        for (Page page : cache.values())
+        {
+            try
+            {
+                page.detached();
+            }
+            catch (Throwable t)
+            {
+                logger.error(String.format("Error detaching page %s: %s", page, InternalUtils.toMessage(t)), t);
+            }
+        }
+    }
+
+    public Page get(String pageName)
+    {
+        String canonical = resolver.canonicalizePageName(pageName);
+
+        Page page = cache.get(canonical);
+
+        if (page == null)
+        {
+            Locale locale = threadLocale.getLocale();
+
+            page = pageSource.getPage(canonical, locale);
+
+            try
+            {
+                page.attached();
+            }
+            catch (Throwable t)
+            {
+                throw new RuntimeException(String.format("Unable to attach page %s (%s): %s", canonical, locale,
+                        InternalUtils.toMessage(t)), t);
+            }
+
+            cache.put(canonical, page);
+        }
+
+        return page;
+    }
+}

Propchange: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/NonPoolingRequestPageCacheImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java?rev=963429&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java (added)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java Mon Jul 12 18:47:41 2010
@@ -0,0 +1,30 @@
+// Copyright 2010 The Apache Software Foundation
+//
+// Licensed 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.tapestry5.internal.services;
+
+import java.util.Locale;
+
+import org.apache.tapestry5.internal.structure.Page;
+
+/**
+ * Access to localized page instances (which are now shared singletons, starting in release 5.2).
+ * This service is a wrapper around {@link PageLoader} that caches the loaded pages.
+ * 
+ * @since 5.2.0
+ */
+public interface PageSource
+{
+    Page getPage(String canonicalPageName, Locale locale);
+}

Propchange: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSource.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java?rev=963429&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java (added)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java Mon Jul 12 18:47:41 2010
@@ -0,0 +1,93 @@
+// Copyright 2010 The Apache Software Foundation
+//
+// Licensed 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.tapestry5.internal.services;
+
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.tapestry5.internal.structure.Page;
+import org.apache.tapestry5.ioc.internal.util.CollectionFactory;
+import org.apache.tapestry5.services.InvalidationListener;
+
+public class PageSourceImpl implements PageSource, InvalidationListener
+{
+    private final PageLoader pageLoader;
+
+    private static final class CachedPageKey
+    {
+        final String pageName;
+
+        final Locale locale;
+
+        public CachedPageKey(String pageName, Locale locale)
+        {
+            this.pageName = pageName;
+            this.locale = locale;
+        }
+
+        public int hashCode()
+        {
+            return 37 * pageName.hashCode() + locale.hashCode();
+        }
+
+        public boolean equals(Object obj)
+        {
+            if (this == obj)
+                return true;
+
+            if (!(obj instanceof CachedPageKey))
+                return false;
+
+            CachedPageKey other = (CachedPageKey) obj;
+
+            return pageName.equals(other.pageName) && locale.equals(other.locale);
+        }
+    }
+
+    private final Map<CachedPageKey, Page> pageCache = CollectionFactory.newConcurrentMap();
+
+    public PageSourceImpl(PageLoader pageLoader)
+    {
+        this.pageLoader = pageLoader;
+    }
+
+    public synchronized void objectWasInvalidated()
+    {
+        pageCache.clear();
+    }
+
+    public Page getPage(String canonicalPageName, Locale locale)
+    {
+        CachedPageKey key = new CachedPageKey(canonicalPageName, locale);
+
+        if (!pageCache.containsKey(key))
+        {
+            // In rare race conditions, we may see the same page loaded multiple times across
+            // different threads. The last built one will "evict" the others from the page cache,
+            // and the earlier ones will be GCed.
+
+            Page page = pageLoader.loadPage(canonicalPageName, locale);
+
+            pageCache.put(key, page);
+        }
+
+        // From good authority (Brian Goetz), this is the best way to ensure that the
+        // loaded page, with all of its semi-mutable construction-time state, is
+        // properly published.
+
+        return pageCache.get(key);
+    }
+
+}

Propchange: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/PageSourceImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCache.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCache.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCache.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCache.java Mon Jul 12 18:47:41 2010
@@ -4,7 +4,7 @@
 // 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
+// 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,
@@ -14,26 +14,34 @@
 
 package org.apache.tapestry5.internal.services;
 
+import org.apache.tapestry5.SymbolConstants;
 import org.apache.tapestry5.internal.structure.Page;
+import org.apache.tapestry5.runtime.PageLifecycleListener;
 
 /**
  * Per-thread service that caches page instances for the duration of the request, and is also responsible for tracking
  * the active page (the page which will ultimately render the response).
  * <p/>
- * Since {@link org.apache.tapestry5.internal.structure.Page} is internal, most user-code should use the {@link
- * org.apache.tapestry5.services.ComponentSource} service instead.
+ * Since {@link org.apache.tapestry5.internal.structure.Page} is internal, most user-code should use the
+ * {@link org.apache.tapestry5.services.ComponentSource} service instead.
+ * <p>
+ * Starting in 5.2, page instances are shared (with externalized mutable state), not pooled, but the cache is still
+ * useful for managing the page's {@linkplain PageLifecycleListener lifecycle}. There are now two different
+ * implementation classes for this single service, selected via {@link SymbolConstants#PAGE_POOL_ENABLED}.
  */
 public interface RequestPageCache
 {
     /**
      * Gets the page via its page name, in the current locale. The logical page name is resolved to a class name, which
-     * is used to obtain the page (from the page pool).  Note that under certain circumstances, a page may have multiple
+     * is used to obtain the page (from the page pool). Note that under certain circumstances, a page may have multiple
      * names (even beyond simple case-insensitivity), and RequestPageCache caches correctly.
-     *
-     * @param pageName the name of the page to retrieve (this is the logical page name, not the fully qualified class
-     *                 name)
+     * 
+     * @param pageName
+     *            the name of the page to retrieve (this is the logical page name, not the fully qualified class
+     *            name)
      * @return a page instance reserved for this request
-     * @throws IllegalArgumentException if the name can not be resolved to a page instance
+     * @throws IllegalArgumentException
+     *             if the name can not be resolved to a page instance
      */
     Page get(String pageName);
 }

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCacheImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCacheImpl.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCacheImpl.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/services/RequestPageCacheImpl.java Mon Jul 12 18:47:41 2010
@@ -4,7 +4,7 @@
 // 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
+// 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,
@@ -22,6 +22,9 @@ import org.apache.tapestry5.services.Com
 
 import java.util.Map;
 
+/**
+ * This is the original implementation, which is built around the {@link PagePool}.
+ */
 public class RequestPageCacheImpl implements RequestPageCache, ThreadCleanupListener
 {
     private final PagePool pagePool;

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/structure/PageImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/structure/PageImpl.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/structure/PageImpl.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/internal/structure/PageImpl.java Mon Jul 12 18:47:41 2010
@@ -14,14 +14,15 @@
 
 package org.apache.tapestry5.internal.structure;
 
+import java.util.List;
+import java.util.Locale;
+
 import org.apache.tapestry5.ComponentResources;
 import org.apache.tapestry5.internal.services.PersistentFieldManager;
 import org.apache.tapestry5.ioc.internal.util.CollectionFactory;
 import org.apache.tapestry5.ioc.internal.util.Defense;
-import org.apache.tapestry5.ioc.internal.util.OneShotLock;
-
-import static org.apache.tapestry5.ioc.internal.util.Defense.notNull;
 import org.apache.tapestry5.ioc.internal.util.InternalUtils;
+import org.apache.tapestry5.ioc.internal.util.OneShotLock;
 import org.apache.tapestry5.ioc.services.PerThreadValue;
 import org.apache.tapestry5.ioc.services.PerthreadManager;
 import org.apache.tapestry5.runtime.Component;
@@ -29,9 +30,6 @@ import org.apache.tapestry5.runtime.Page
 import org.apache.tapestry5.services.PersistentFieldBundle;
 import org.slf4j.Logger;
 
-import java.util.List;
-import java.util.Locale;
-
 public class PageImpl implements Page
 {
     private final String name;
@@ -57,8 +55,18 @@ public class PageImpl implements Page
      * first needed,
      * discarded at the end of the request.
      */
-    private PersistentFieldBundle fieldBundle;
+    private final PerThreadValue<PersistentFieldBundle> fieldBundle;
 
+    /**
+     * @param name
+     *            canonicalized page name
+     * @param locale
+     *            locale for page and all components
+     * @param persistentFieldManager
+     *            for access to cross-request persistent values
+     * @param perThreadManager
+     *            for managing per-request mutable state
+     */
     public PageImpl(String name, Locale locale, PersistentFieldManager persistentFieldManager,
             PerthreadManager perThreadManager)
     {
@@ -67,6 +75,7 @@ public class PageImpl implements Page
         this.persistentFieldManager = persistentFieldManager;
 
         dirtyCount = perThreadManager.createValue("PageDirtyCount:" + name);
+        fieldBundle = perThreadManager.createValue("PersistentFieldBundle:" + name);
     }
 
     @Override
@@ -77,7 +86,7 @@ public class PageImpl implements Page
 
     public ComponentPageElement getComponentElementByNestedId(String nestedId)
     {
-        notNull(nestedId, "nestedId");
+        Defense.notNull(nestedId, "nestedId");
 
         // TODO: Especially with the addition of all the caseless logic, and with respect to how
         // forms are implemented, it may be worthwhile to cache the key to element mapping. I think
@@ -147,8 +156,6 @@ public class PageImpl implements Page
             }
         }
 
-        fieldBundle = null;
-
         return result;
     }
 
@@ -160,7 +167,7 @@ public class PageImpl implements Page
             listener.containingPageDidLoad();
 
         loadComplete = true;
-        
+
         lock.lock();
     }
 
@@ -191,10 +198,10 @@ public class PageImpl implements Page
 
     public Object getFieldChange(String nestedId, String fieldName)
     {
-        if (fieldBundle == null)
-            fieldBundle = persistentFieldManager.gatherChanges(name);
+        if (!fieldBundle.exists())
+            fieldBundle.set(persistentFieldManager.gatherChanges(name));
 
-        return fieldBundle.getValue(nestedId, fieldName);
+        return fieldBundle.get().getValue(nestedId, fieldName);
     }
 
     public void decrementDirtyCount()
@@ -224,7 +231,7 @@ public class PageImpl implements Page
     public void addResetListener(PageResetListener listener)
     {
         Defense.notNull(listener, "listener");
-        
+
         lock.check();
 
         resetListeners.add(listener);

Modified: tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/services/TapestryModule.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/services/TapestryModule.java?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/services/TapestryModule.java (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/main/java/org/apache/tapestry5/services/TapestryModule.java Mon Jul 12 18:47:41 2010
@@ -516,8 +516,8 @@ public final class TapestryModule
             configuration.add(folder, new ClasspathAssetRequestHandler(streamer, assetResourceLocator, path));
         }
 
-        configuration.add(RequestConstants.CONTEXT_FOLDER, new ContextAssetRequestHandler(streamer, contextAssetFactory
-                .getRootResource()));
+        configuration.add(RequestConstants.CONTEXT_FOLDER,
+                new ContextAssetRequestHandler(streamer, contextAssetFactory.getRootResource()));
 
         configuration.add(RequestConstants.STACK_FOLDER, stackAssetRequestHandler);
 
@@ -609,7 +609,8 @@ public final class TapestryModule
         configuration.add("MixinAfter", new MixinAfterWorker());
         configuration.add("Component", new ComponentWorker(resolver));
         configuration.add("Mixin", new MixinWorker(resolver));
-        configuration.addInstance("ActivationRequestParameter", ActivationRequestParameterWorker.class, "before:OnEvent");
+        configuration.addInstance("ActivationRequestParameter", ActivationRequestParameterWorker.class,
+                "before:OnEvent");
         configuration.addInstance("OnEvent", OnEventWorker.class);
         configuration.add("SupportsInformalParameters", new SupportsInformalParametersWorker());
         configuration.addInstance("InjectPage", InjectPageWorker.class);
@@ -1333,8 +1334,8 @@ public final class TapestryModule
 
     ClasspathURLConverter classpathURLConverter)
     {
-        ValidationMessagesSourceImpl service = new ValidationMessagesSourceImpl(configuration, classpathAssetFactory
-                .getRootResource(), parser, componentMessagesSource, classpathURLConverter);
+        ValidationMessagesSourceImpl service = new ValidationMessagesSourceImpl(configuration,
+                classpathAssetFactory.getRootResource(), parser, componentMessagesSource, classpathURLConverter);
         updateListenerHub.addUpdateListener(service);
 
         return service;
@@ -1546,8 +1547,8 @@ public final class TapestryModule
             TranslatorAlternatesSource alternatesSource, @ComponentClasses
             InvalidationEventHub hub)
     {
-        TranslatorSourceImpl service = new TranslatorSourceImpl(configuration, alternatesSource
-                .getTranslatorAlternates());
+        TranslatorSourceImpl service = new TranslatorSourceImpl(configuration,
+                alternatesSource.getTranslatorAlternates());
 
         hub.addInvalidationListener(service);
 
@@ -2371,13 +2372,12 @@ public final class TapestryModule
         configuration.add(SymbolConstants.FILE_CHECK_UPDATE_TIMEOUT, "50 ms");
 
         // This should be overridden for particular applications. These are the
-        // locales for
-        // which we have (at least some) localized messages.
+        // locales for which we have (at least some) localized messages.
         configuration.add(SymbolConstants.SUPPORTED_LOCALES,
                 "en,it,es,zh_CN,pt_PT,de,ru,hr,fi_FI,sv_SE,fr_FR,da,pt_BR,ja,el,bg,no_NB");
 
-        configuration.add(SymbolConstants.TAPESTRY_VERSION, VersionUtils
-                .readVersionNumber("META-INF/maven/org.apache.tapestry/tapestry-core/pom.properties"));
+        configuration.add(SymbolConstants.TAPESTRY_VERSION,
+                VersionUtils.readVersionNumber("META-INF/maven/org.apache.tapestry/tapestry-core/pom.properties"));
 
         configuration.add(SymbolConstants.COOKIE_MAX_AGE, "7 d");
 
@@ -2392,6 +2392,7 @@ public final class TapestryModule
         configuration.add(SymbolConstants.PAGE_POOL_SOFT_WAIT, "10 ms");
         configuration.add(SymbolConstants.PAGE_POOL_HARD_LIMIT, "20");
         configuration.add(SymbolConstants.PAGE_POOL_ACTIVE_WINDOW, "10 m");
+        configuration.add(SymbolConstants.PAGE_POOL_ENABLED, "false");
 
         configuration.add(SymbolConstants.SUPPRESS_REDIRECT_FROM_ACTION_REQUESTS, "false");
 
@@ -2434,8 +2435,8 @@ public final class TapestryModule
 
         configuration.add(SymbolConstants.CHARSET, "UTF-8");
 
-        configuration.add(SymbolConstants.APPLICATION_CATALOG, String.format("context:WEB-INF/${%s}.properties",
-                InternalSymbols.APP_NAME));
+        configuration.add(SymbolConstants.APPLICATION_CATALOG,
+                String.format("context:WEB-INF/${%s}.properties", InternalSymbols.APP_NAME));
 
         configuration.add(SymbolConstants.EXCEPTION_REPORT_PAGE, "ExceptionReport");
 

Modified: tapestry/tapestry5/trunk/tapestry-core/src/test/resources/org/apache/tapestry5/integration/reload/web.xml
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-core/src/test/resources/org/apache/tapestry5/integration/reload/web.xml?rev=963429&r1=963428&r2=963429&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-core/src/test/resources/org/apache/tapestry5/integration/reload/web.xml (original)
+++ tapestry/tapestry5/trunk/tapestry-core/src/test/resources/org/apache/tapestry5/integration/reload/web.xml Mon Jul 12 18:47:41 2010
@@ -24,6 +24,10 @@
         <param-name>tapestry.app-package</param-name>
         <param-value>org.apache.tapestry5.integration.reload</param-value>
     </context-param>
+    <context-param>
+      <param-name>tapestry.page-pool-enabled</param-name>
+      <param-value>true</param-value>
+    </context-param>
     <filter>
         <filter-name>app</filter-name>
         <filter-class>org.apache.tapestry5.TapestryFilter</filter-class>