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 2008/04/20 20:56:24 UTC

svn commit: r649954 - in /tapestry/tapestry5/trunk: tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/ tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/ tapestry-hibernate/src/test/java/org/apache/tapestry/internal/h...

Author: hlship
Date: Sun Apr 20 11:56:22 2008
New Revision: 649954

URL: http://svn.apache.org/viewvc?rev=649954&view=rev
Log:
TAPESTRY-2365: It should be possible to create decorators without directly writing Javassist code

Added:
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/Invocation.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/MethodAdvice.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AbstractInvocation.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectDecoratorImpl.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImpl.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectDecorator.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectInterceptorBuilder.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AbstractInvocationTest.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImplTest.java
Removed:
    tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/ServiceLoggerTest.java
Modified:
    tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/HibernateTransactionDecorator.java
    tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImpl.java
    tapestry/tapestry5/trunk/tapestry-hibernate/src/test/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImplTest.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImpl.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/ServiceLogger.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/TapestryIOCModule.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/site/apt/decorator.apt
    tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImplTest.java
    tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/util/TimeIntervalTest.java

Modified: tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/HibernateTransactionDecorator.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/HibernateTransactionDecorator.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/HibernateTransactionDecorator.java (original)
+++ tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/hibernate/HibernateTransactionDecorator.java Sun Apr 20 11:56:22 2008
@@ -24,7 +24,7 @@
 public interface HibernateTransactionDecorator
 {
     /**
-     * Builds an  transaction interceptor instance.
+     * Builds a transaction interceptor instance around the delegate.
      *
      * @param <T>
      * @param serviceInterface interface implemented by the delegate

Modified: tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImpl.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImpl.java (original)
+++ tapestry/tapestry5/trunk/tapestry-hibernate/src/main/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImpl.java Sun Apr 20 11:56:22 2008
@@ -17,134 +17,72 @@
 import org.apache.tapestry.hibernate.HibernateSessionManager;
 import org.apache.tapestry.hibernate.HibernateTransactionDecorator;
 import org.apache.tapestry.hibernate.annotations.CommitAfter;
-import org.apache.tapestry.ioc.services.*;
-import org.apache.tapestry.ioc.util.BodyBuilder;
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.MethodAdvice;
+import org.apache.tapestry.ioc.internal.util.Defense;
+import org.apache.tapestry.ioc.services.AspectDecorator;
+import org.apache.tapestry.ioc.services.AspectInterceptorBuilder;
 
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 
 public class HibernateTransactionDecoratorImpl implements HibernateTransactionDecorator
 {
-    private ClassFactory _classFactory;
+    private final AspectDecorator _aspectDecorator;
 
-    private HibernateSessionManager _manager;
+    private final HibernateSessionManager _manager;
 
-    public HibernateTransactionDecoratorImpl(
-            // Use the IOC's ClassFactory, not the component ClassFactory
-            @Builtin ClassFactory classFactory,
-
-            HibernateSessionManager manager)
-    {
-        _classFactory = classFactory;
-        _manager = manager;
-    }
-
-    public <T> T build(Class<T> serviceInterface, T delegate, String serviceId)
-    {
-        Class interceptorClass = createInterceptorClass(serviceInterface, serviceId);
-        Constructor cc = interceptorClass.getConstructors()[0];
-
-        Object interceptor = null;
-        Throwable fail = null;
-
-        try
-        {
-            interceptor = cc.newInstance(delegate, _manager);
-        }
-        catch (InvocationTargetException ite)
-        {
-            fail = ite.getTargetException();
-        }
-        catch (Exception ex)
-        {
-            fail = ex;
-        }
-
-        if (fail != null) throw new RuntimeException(fail);
-
-        return serviceInterface.cast(interceptor);
-    }
-
-    private Class createInterceptorClass(Class serviceInterface, String serviceId)
-    {
-        ClassFab cf = _classFactory.newClass(serviceInterface);
-
-        cf.addField("_delegate", Modifier.PRIVATE | Modifier.FINAL, serviceInterface);
-        cf.addField("_manager", Modifier.PRIVATE | Modifier.FINAL, HibernateSessionManager.class);
-
-        cf.addConstructor(new Class[] { serviceInterface, HibernateSessionManager.class }, null,
-                          "{ _delegate = $1; _manager = $2; }");
-
-        addMethods(cf, serviceInterface, serviceId);
-
-        return cf.createClass();
-    }
-
-    private void addMethods(ClassFab cf, Class serviceInterface, String serviceId)
+    /**
+     * The rules for advice are the same for any method: commit on success or checked exception, abort on thrown
+     * exception ... so we can use a single shared advice object.
+     */
+    private final MethodAdvice _advice = new MethodAdvice()
     {
-        Method[] methods = serviceInterface.getMethods();
-
-        Method toString = null;
-
-        for (Method method : methods)
+        public void advise(Invocation invocation)
         {
-            CommitAfter annotation = method.getAnnotation(CommitAfter.class);
+            try
+            {
+                invocation.proceed();
+            }
+            catch (RuntimeException ex)
+            {
+                _manager.abort();
 
-            MethodSignature signature = new MethodSignature(method);
+                throw ex;
+            }
 
-            addMethod(cf, signature, annotation != null);
+            // For success or checked exception, commit the transaction.
 
-            if (ClassFabUtils.isToString(method))
-            {
-                toString = method;
-            }
+            _manager.commit();
         }
+    };
 
-        if (toString == null)
-            cf.addToString(HibernateMessages.commitTransactionInterceptor(serviceId, serviceInterface));
+    public HibernateTransactionDecoratorImpl(AspectDecorator aspectDecorator, HibernateSessionManager manager)
+    {
+        _aspectDecorator = aspectDecorator;
 
+        _manager = manager;
     }
 
-    private void addMethod(ClassFab cf, MethodSignature signature, boolean commit)
+    public <T> T build(Class<T> serviceInterface, T delegate, String serviceId)
     {
-        Class returnType = signature.getReturnType();
-        boolean isVoid = returnType.equals(void.class);
-
-        BodyBuilder builder = new BodyBuilder().begin();
-
-        if (commit) builder.addln("try").begin();
-
-        if (!isVoid) builder.add("%s result = ", ClassFabUtils.toJavaClassName(returnType));
-
-        builder.addln("_delegate.%s($$);", signature.getName());
+        Defense.notNull(serviceInterface, "serviceInterface");
+        Defense.notNull(delegate, "delegate");
+        Defense.notBlank(serviceId, "serviceId");
 
-        if (commit)
-            builder.addln("_manager.commit();");
+        String description = String.format("<Hibernate Transaction interceptor for %s(%s)>",
+                                           serviceId,
+                                           serviceInterface.getName());
 
-        if (!isVoid) builder.addln("return result;");
+        AspectInterceptorBuilder<T> builder = _aspectDecorator.createBuilder(serviceInterface, delegate, description);
 
-        if (commit)
+        for (Method m : serviceInterface.getMethods())
         {
-            builder.end(); // try
-
-            // Now to handle exceptions. All runtime exceptions cause an abort.
-
-            builder.addln("catch (RuntimeException ex)");
-            builder.begin().addln("_manager.abort(); throw ex;").end();
-
-            // Next, each exception in the signature.  Declared exceptions
-            // commit the transaction just like success.
-
-            for (Class exceptionType : signature.getExceptionTypes())
+            if (m.getAnnotation(CommitAfter.class) != null)
             {
-                builder.addln("catch (%s ex)", exceptionType.getName());
-                builder.begin().addln("_manager.commit(); throw ex;").end();
+                builder.adviseMethod(m, _advice);
             }
         }
 
-        cf.addMethod(Modifier.PUBLIC, signature, builder.end().toString());
+        return builder.build();
     }
-
 }

Modified: tapestry/tapestry5/trunk/tapestry-hibernate/src/test/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImplTest.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-hibernate/src/test/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImplTest.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-hibernate/src/test/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImplTest.java (original)
+++ tapestry/tapestry5/trunk/tapestry-hibernate/src/test/java/org/apache/tapestry/internal/hibernate/HibernateTransactionDecoratorImplTest.java Sun Apr 20 11:56:22 2008
@@ -17,8 +17,12 @@
 import org.apache.tapestry.hibernate.HibernateSessionManager;
 import org.apache.tapestry.hibernate.HibernateTransactionDecorator;
 import org.apache.tapestry.hibernate.annotations.CommitAfter;
-import org.apache.tapestry.ioc.internal.services.ClassFactoryImpl;
+import org.apache.tapestry.ioc.IOCUtilities;
+import org.apache.tapestry.ioc.Registry;
+import org.apache.tapestry.ioc.services.AspectDecorator;
 import org.apache.tapestry.test.TapestryTestCase;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 
 import java.sql.SQLException;
@@ -26,6 +30,26 @@
 @SuppressWarnings({ "ThrowableInstanceNeverThrown" })
 public class HibernateTransactionDecoratorImplTest extends TapestryTestCase
 {
+    private Registry _registry;
+
+    private AspectDecorator _aspectDecorator;
+
+    @BeforeClass
+    public void setup()
+    {
+        _registry = IOCUtilities.buildDefaultRegistry();
+
+        _aspectDecorator = _registry.getService(AspectDecorator.class);
+    }
+
+
+    @AfterClass
+    public void shutdown()
+    {
+        _aspectDecorator = null;
+        _registry.shutdown();
+        _registry = null;
+    }
 
     @Test
     public void undecorated()
@@ -179,7 +203,7 @@
 
     private HibernateTransactionDecorator newHibernateSessionManagerDecorator(HibernateSessionManager manager)
     {
-        return new HibernateTransactionDecoratorImpl(new ClassFactoryImpl(), manager);
+        return new HibernateTransactionDecoratorImpl(_aspectDecorator, manager);
     }
 
     private void assertToString(VoidService interceptor)

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/Invocation.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/Invocation.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/Invocation.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/Invocation.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,93 @@
+// Copyright 2008 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.tapestry.ioc;
+
+/**
+ * A method invocation passed to an {@link MethodAdvice}.
+ */
+public interface Invocation
+{
+    /**
+     * Returns the name of the method being invoked.
+     */
+    String getMethodName();
+
+    /**
+     * Returns the type of the method result, which may be a primtive type (i.e., int.class) or even void (void.class).
+     */
+    Class getResultType();
+
+    /**
+     * Returns the number of parameters passed to the method.
+     */
+    int getParameterCount();
+
+    /**
+     * Returns the type of the parameter at the index.
+     */
+    Class getParameterType(int index);
+
+    /**
+     * Returns the indicated parameter (may return null if the parameter is null).
+     */
+    Object getParameter(int index);
+
+    /**
+     * Replaces a parameter in the invocation.
+     *
+     * @param index        of parameter to update
+     * @param newParameter new parameter value (may be null)
+     */
+    void override(int index, Object newParameter);
+
+    /**
+     * Processed with the invocation.  If the invocation results in a <em>runtime</em> exception, that is thrown.
+     */
+    void proceed();
+
+    /**
+     * If true, then the proceeded invocation threw a checked exception.
+     */
+    boolean isFail();
+
+    /**
+     * After invoking {@link #proceed()}, used to obtain the thrown (checked) exception, if assignable to the provided
+     * type.
+     *
+     * @param throwableClass the type of exception to match
+     * @return the exception, if the proceeded invocation threw a checked exception, and the exception is assignable to
+     *         the provided type.
+     */
+    <T extends Throwable> T getThrown(Class<T> throwableClass);
+
+    /**
+     * Overrides the thrown exception. The passed exception should be a checked exception of the method. Note that for
+     * runtime exceptions, or even {@link Error}s, those can just be thrown. Sets the fail flag.
+     *
+     * @param thrown
+     * @throws IllegalArgumentException if thrown is null, or not a declared exception of the method
+     */
+    void overrideThrown(Exception thrown);
+
+    /**
+     * The return value after {@link #proceed()}, which may be null.
+     */
+    Object getResult();
+
+    /**
+     * Overrides the result. Clears the thrown exception (if any).
+     */
+    void overrideResult(Object newResult);
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/MethodAdvice.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/MethodAdvice.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/MethodAdvice.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/MethodAdvice.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,35 @@
+// Copyright 2008 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.tapestry.ioc;
+
+/**
+ * For Tapestry IoC, Aspects provide a limited amount of advise, i.e., advising method invocations. That's the only join
+ * point available (method invocations of service interface methods); full AOP systems such as AspectJ can do much, much
+ * such as advising field access and even object construction.
+ *
+ * @see org.apache.tapestry.ioc.services.AspectDecorator
+ */
+public interface MethodAdvice
+{
+    /**
+     * Allows the Aspect to advise the invocation.  The Aspect is free to inspect and even replace parameters. Most
+     * Aspects will then invoke {@link org.apache.tapestry.ioc.Invocation#proceed()}.  The Aspect may then inspect and
+     * replace any checked thrown exceptions. Some Aspects (for example, caching) may selectively decide to bypass the
+     * invocation entirely, and instead invoke some other method or otherwise set a return value or thrown exception.
+     *
+     * @param invocation to advise
+     */
+    void advise(Invocation invocation);
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AbstractInvocation.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AbstractInvocation.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AbstractInvocation.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AbstractInvocation.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,106 @@
+// Copyright 2008 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.tapestry.ioc.internal.services;
+
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.internal.util.Defense;
+
+import java.lang.reflect.Method;
+
+/**
+ * Base class for {@link org.apache.tapestry.ioc.Invocation}, which is extended with a dynamically generated class
+ * generated for each individual class and method.
+ */
+public abstract class AbstractInvocation implements Invocation
+{
+    private final Method _method;
+
+    private Throwable _thrown;
+
+    private Object _result;
+
+    @Override
+    public String toString()
+    {
+        return String.format("Invocation[%s]", _method);
+    }
+
+    protected AbstractInvocation(Method method)
+    {
+        _method = method;
+    }
+
+    public String getMethodName()
+    {
+        return _method.getName();
+    }
+
+    public Class getResultType()
+    {
+        return _method.getReturnType();
+    }
+
+    public int getParameterCount()
+    {
+        return _method.getParameterTypes().length;
+    }
+
+    public Class getParameterType(int index)
+    {
+        return _method.getParameterTypes()[index];
+    }
+
+    public boolean isFail()
+    {
+        return _thrown != null;
+    }
+
+    public <T extends Throwable> T getThrown(Class<T> throwableClass)
+    {
+        Defense.notNull(throwableClass, "throwableClass");
+
+        if (throwableClass.isInstance(_thrown)) return throwableClass.cast(_thrown);
+
+        return null;
+    }
+
+    public void overrideThrown(Exception thrown)
+    {
+        Defense.notNull(thrown, "thrown");
+
+        for (Class t : _method.getExceptionTypes())
+        {
+            if (t.isInstance(thrown))
+            {
+                _thrown = thrown;
+                return;
+            }
+        }
+
+        throw new IllegalArgumentException(String.format("Exception %s is not a declared exception of method %s.",
+                                                         thrown.getClass().getName(), _method));
+    }
+
+    public Object getResult()
+    {
+        return _result;
+    }
+
+    public void overrideResult(Object newResult)
+    {
+        _result = newResult;
+        _thrown = null;
+    }
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectDecoratorImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectDecoratorImpl.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectDecoratorImpl.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectDecoratorImpl.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,78 @@
+// Copyright 2008 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.tapestry.ioc.internal.services;
+
+import org.apache.tapestry.ioc.MethodAdvice;
+import org.apache.tapestry.ioc.internal.util.Defense;
+import org.apache.tapestry.ioc.services.AspectDecorator;
+import org.apache.tapestry.ioc.services.AspectInterceptorBuilder;
+import org.apache.tapestry.ioc.services.Builtin;
+import org.apache.tapestry.ioc.services.ClassFactory;
+
+import java.lang.reflect.Method;
+
+public class AspectDecoratorImpl implements AspectDecorator
+{
+    private final ClassFactory _classFactory;
+
+    public AspectDecoratorImpl(@Builtin ClassFactory classFactory)
+    {
+        _classFactory = classFactory;
+    }
+
+    public <T> T build(Class<T> serviceInterface, T delegate, MethodAdvice advice, String description)
+    {
+        Defense.notNull(advice, "advice");
+
+        AspectInterceptorBuilder<T> builder = createBuilder(serviceInterface, delegate, description);
+
+        // Use the same advice for all methods.
+
+        for (Method m : serviceInterface.getMethods())
+            builder.adviseMethod(m, advice);
+
+        return builder.build();
+    }
+
+    public <T> AspectInterceptorBuilder<T> createBuilder(final Class<T> serviceInterface, final T delegate,
+                                                         final String description)
+    {
+        Defense.notNull(serviceInterface, "serviceInterface");
+        Defense.notNull(delegate, "delegate");
+        Defense.notBlank(description, "description");
+
+        // Defer creating the real builder until a method gets advised.  If no method is advised then
+        // the delegate can be used unchanged.
+
+        return new AspectInterceptorBuilder<T>()
+        {
+            private AspectInterceptorBuilder<T> _builder;
+
+            public void adviseMethod(Method method, MethodAdvice advice)
+            {
+                if (_builder == null)
+                    _builder = new AspectInterceptorBuilderImpl<T>(_classFactory, serviceInterface, delegate,
+                                                                   description);
+
+                _builder.adviseMethod(method, advice);
+            }
+
+            public T build()
+            {
+                return _builder == null ? delegate : _builder.build();
+            }
+        };
+    }
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImpl.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImpl.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImpl.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,404 @@
+// Copyright 2008 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.tapestry.ioc.internal.services;
+
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.MethodAdvice;
+import org.apache.tapestry.ioc.internal.util.CollectionFactory;
+import org.apache.tapestry.ioc.internal.util.Defense;
+import org.apache.tapestry.ioc.internal.util.OneShotLock;
+import org.apache.tapestry.ioc.services.*;
+import org.apache.tapestry.ioc.util.BodyBuilder;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * TODO: Perhaps some caching, would be useful if the same interface has multiple Aspects of interception.
+ */
+public class AspectInterceptorBuilderImpl<T> implements AspectInterceptorBuilder<T>
+{
+    private static final String PARAMETER_FIELD = "_p";
+
+    private static final int PRIVATE_FINAL = Modifier.PRIVATE | Modifier.FINAL;
+
+    private final ClassFactory _classFactory;
+
+    private final Class<T> _serviceInterface;
+
+    private final ClassFab _interceptorFab;
+
+    private final String _delegateFieldName;
+
+    private final String _description;
+
+    private boolean _sawToString;
+    private final OneShotLock _lock = new OneShotLock();
+
+    private static class Injection
+    {
+        final String _fieldName;
+        final Class _fieldType;
+        final Object _injectedValue;
+
+        private Injection(String fieldName, Class fieldType, Object injectedValue)
+        {
+            _fieldName = fieldName;
+            _fieldType = fieldType;
+            _injectedValue = injectedValue;
+        }
+    }
+
+    private final List<Injection> _injections = CollectionFactory.newList();
+
+    private final Map<Object, Injection> _objectToInjection = CollectionFactory.newMap();
+
+    private final Set<Method> _remainingMethods = CollectionFactory.newSet();
+
+    private final Set<Method> _advisedMethods = CollectionFactory.newSet();
+
+    public AspectInterceptorBuilderImpl(ClassFactory classFactory, Class<T> serviceInterface, T delegate,
+                                        String description)
+    {
+        _classFactory = classFactory;
+        _serviceInterface = serviceInterface;
+        _description = description;
+
+        _interceptorFab = _classFactory.newClass(serviceInterface);
+
+        _delegateFieldName = inject(_serviceInterface, delegate);
+
+        _remainingMethods.addAll(Arrays.asList(serviceInterface.getMethods()));
+    }
+
+    public void adviseMethod(Method method, MethodAdvice advice)
+    {
+        Defense.notNull(method, "method");
+        Defense.notNull(advice, "advice");
+
+        _lock.check();
+
+        if (_advisedMethods.contains(method))
+            throw new IllegalArgumentException(String.format("Method %s has already been advised.", method));
+
+        if (!_remainingMethods.contains(method))
+            throw new IllegalArgumentException(
+                    String.format("Method %s is not defined for interface %s.", method, _serviceInterface));
+
+        _sawToString |= ClassFabUtils.isToString(method);
+
+        String invocationClassName = createInvocationClass(method);
+
+        BodyBuilder builder = new BodyBuilder().begin();
+
+        String methodFieldName = inject(Method.class, method);
+        String aspectFieldName = inject(MethodAdvice.class, advice);
+
+        builder.addln("%s invocation = new %s(%s, %s, $$);", Invocation.class.getName(), invocationClassName,
+                      methodFieldName, _delegateFieldName);
+
+        builder.addln("%s.advise(invocation);", aspectFieldName);
+
+        Class[] exceptionTypes = method.getExceptionTypes();
+
+        builder.addln("if (invocation.isFail())").begin();
+
+        for (Class exceptionType : exceptionTypes)
+        {
+            String name = exceptionType.getSimpleName().toLowerCase();
+
+            String exceptionTypeFieldName = inject(Class.class, exceptionType);
+
+            builder.addln("%s %s = (%s) invocation.getThrown(%s);", exceptionType.getName(), name,
+                          exceptionType.getName(), exceptionTypeFieldName);
+            builder.addln("if (%s != null) throw %s;", name, name);
+        }
+
+        builder.addln(
+                "throw new IllegalStateException(\"Impossible exception thrown from intercepted invocation.\");");
+
+        builder.end(); // if fail
+
+        builder.addln("return ($r) invocation.getResult();");
+
+        builder.end();
+
+        _interceptorFab.addMethod(Modifier.PUBLIC, new MethodSignature(method), builder.toString());
+
+        _remainingMethods.remove(method);
+        _advisedMethods.add(method);
+    }
+
+    private String createInvocationClass(Method method)
+    {
+        String baseName = _serviceInterface.getSimpleName() + "$" + method.getName();
+        String className = ClassFabUtils.generateClassName(baseName);
+
+        ClassFab invocationFab = _classFactory.newClass(className, AbstractInvocation.class);
+
+        List<Class> constructorTypes = CollectionFactory.newList();
+
+        // The first two parameters are fixed:
+
+        constructorTypes.add(Method.class); // And passed up to the super class
+
+        invocationFab.addField("_delegate", PRIVATE_FINAL, _serviceInterface);
+        constructorTypes.add(_serviceInterface);
+
+        BodyBuilder constructorBuilder = new BodyBuilder().begin().addln("super($1);").addln("_delegate = $2;");
+
+        for (int i = 0; i < method.getParameterTypes().length; i++)
+        {
+            Class type = method.getParameterTypes()[i];
+
+            String name = PARAMETER_FIELD + i;
+
+            invocationFab.addField(name, type);
+
+            constructorTypes.add(type);
+
+            // $0 is this
+            // $1 is Method
+            // $2 is delegate
+            // $3 is first method parameter ...
+
+            constructorBuilder.addln("%s = $%d;", name, i + 3);
+        }
+
+        addProceed(method, invocationFab);
+        addGetParameter(method, invocationFab);
+        addOverride(method, invocationFab);
+
+        constructorBuilder.end(); // constructor
+
+        Class[] typesArray = constructorTypes.toArray(new Class[constructorTypes.size()]);
+
+        invocationFab.addConstructor(typesArray, null, constructorBuilder.toString());
+
+        invocationFab.createClass();
+
+        return className;
+    }
+
+    private void addProceed(Method method, ClassFab fab)
+    {
+        Class returnType = method.getReturnType();
+        Class[] exceptionTypes = method.getExceptionTypes();
+
+        boolean isNonVoid = !returnType.equals(void.class);
+        boolean hasChecked = exceptionTypes.length > 0;
+
+        BodyBuilder builder = new BodyBuilder().begin();
+
+        if (hasChecked) builder.addln("try").begin();
+
+        if (isNonVoid)
+            builder.add("%s result = ", returnType.getName());
+
+        builder.add("_delegate.%s(", method.getName());
+
+        for (int i = 0; i < method.getParameterTypes().length; i++)
+        {
+            if (i > 0) builder.add(", ");
+
+            builder.add(PARAMETER_FIELD + i);
+        }
+
+        builder.addln(");"); // Call on delegate
+
+        if (isNonVoid)
+        {
+            builder.add("overrideResult(($w) result);");
+        }
+
+        if (hasChecked)
+        {
+            builder.end();   // try
+
+            for (Class exception : exceptionTypes)
+            {
+                builder.addln("catch (%s ex) { overrideThrown(ex); }", exception.getName());
+            }
+        }
+
+        builder.end(); // method
+
+        MethodSignature sig = new MethodSignature(void.class, "proceed", null, null);
+
+        fab.addMethod(Modifier.PUBLIC, sig, builder.toString());
+    }
+
+    private void addGetParameter(Method method, ClassFab fab)
+    {
+        Class[] parameterTypes = method.getParameterTypes();
+
+        BodyBuilder builder = new BodyBuilder().begin();
+
+        builder.addln("switch ($1)").begin();
+
+        for (int i = 0; i < parameterTypes.length; i++)
+        {
+            // ($w) will wrap a primitive as a wrapper type
+            builder.addln("case %d: return ($w) %s%d;", i, PARAMETER_FIELD, i);
+        }
+
+        builder.addln("default: throw new IllegalArgumentException(\"Parameter index out of range.\");");
+
+        builder.end().end(); // switch and method
+
+        fab.addMethod(Modifier.PUBLIC,
+                      new MethodSignature(Object.class, "getParameter", new Class[] { int.class }, null),
+                      builder.toString());
+
+    }
+
+    private void addOverride(Method method, ClassFab fab)
+    {
+        Class[] parameterTypes = method.getParameterTypes();
+
+        BodyBuilder builder = new BodyBuilder().begin();
+
+        builder.addln("switch ($1)").begin();
+
+        for (int i = 0; i < parameterTypes.length; i++)
+        {
+            Class type = parameterTypes[i];
+
+            builder.add("case %d: %s%d = ", i, PARAMETER_FIELD, i);
+
+            if (type.isPrimitive())
+            {
+                Class wrapperType = ClassFabUtils.getWrapperType(type);
+
+                builder.add("((%s)$2).%s()", wrapperType.getName(),
+                            ClassFabUtils.getUnwrapMethodName(wrapperType));
+            }
+            else
+            {
+                builder.add("(%s)$2", ClassFabUtils.toJavaClassName(type));
+            }
+
+            builder.addln(";");
+
+            builder.addln("return;");
+        }
+
+        builder.addln("default: throw new IllegalArgumentException(\"Parameter index out of range.\");");
+
+        builder.end().end(); // switch and method
+
+        fab.addMethod(Modifier.PUBLIC,
+                      new MethodSignature(void.class, "override", new Class[] { int.class, Object.class }, null),
+                      builder.toString());
+    }
+
+    public T build()
+    {
+        _lock.lock();
+
+        // Hit all the methods that haven't been referenced so far.
+
+        addPassthruMethods();
+
+        // And if we haven't seend a toString(), we can add it now.
+
+        if (!_sawToString)
+            _interceptorFab.addToString(_description);
+
+        Object[] parameters = createConstructor();
+
+        try
+        {
+            Class c = _interceptorFab.createClass();
+
+            // There's only ever the one constructor.
+
+            Constructor cc = c.getConstructors()[0];
+
+            Object interceptor = cc.newInstance(parameters);
+
+            return _serviceInterface.cast(interceptor);
+        }
+        catch (Exception ex)
+        {
+            throw new RuntimeException(ex);
+        }
+    }
+
+    private Object[] createConstructor()
+    {
+        // Time to add the constructor.
+
+        Class[] parameterTypes = new Class[_injections.size()];
+        Object[] parameters = new Object[_injections.size()];
+
+        BodyBuilder builder = new BodyBuilder().begin();
+
+        for (int i = 0; i < _injections.size(); i++)
+        {
+            Injection injection = _injections.get(i);
+
+            builder.addln("%s = $%d;", injection._fieldName, i + 1);
+
+            parameterTypes[i] = injection._fieldType;
+            parameters[i] = injection._injectedValue;
+        }
+
+        builder.end();
+
+        _interceptorFab.addConstructor(parameterTypes, null, builder.toString());
+
+        return parameters;
+    }
+
+    private void addPassthruMethods()
+    {
+        for (Method m : _remainingMethods)
+        {
+            _sawToString |= ClassFabUtils.isToString(m);
+
+            MethodSignature sig = new MethodSignature(m);
+
+            String body = String.format("return ($r) %s.%s($$);", _delegateFieldName, m.getName());
+
+            _interceptorFab.addMethod(Modifier.PUBLIC, sig, body);
+        }
+    }
+
+    private <T> String inject(Class<T> fieldType, T injectedValue)
+    {
+        Injection injection = _objectToInjection.get(injectedValue);
+
+        if (injection == null)
+        {
+            String name = "_" + fieldType.getSimpleName().toLowerCase() + "_" + _injections.size();
+
+            _interceptorFab.addField(name, PRIVATE_FINAL, fieldType);
+
+            injection = new Injection(name, fieldType, injectedValue);
+
+            _injections.add(injection);
+            _objectToInjection.put(injectedValue, injection);
+        }
+
+        return injection._fieldName;
+    }
+
+}

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImpl.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImpl.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImpl.java (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImpl.java Sun Apr 20 11:56:22 2008
@@ -1,4 +1,4 @@
-// Copyright 2006, 2007 The Apache Software Foundation
+// Copyright 2006, 2007, 2008 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.
@@ -14,146 +14,66 @@
 
 package org.apache.tapestry.ioc.internal.services;
 
-import org.apache.tapestry.ioc.services.*;
-import static org.apache.tapestry.ioc.services.ClassFabUtils.toJavaClassName;
-import org.apache.tapestry.ioc.util.BodyBuilder;
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.MethodAdvice;
+import org.apache.tapestry.ioc.services.AspectDecorator;
+import org.apache.tapestry.ioc.services.ExceptionTracker;
+import org.apache.tapestry.ioc.services.LoggingDecorator;
 import org.slf4j.Logger;
 
-import static java.lang.String.format;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Modifier;
-
 public class LoggingDecoratorImpl implements LoggingDecorator
 {
-    private final ClassFactory _classFactory;
+    private final AspectDecorator _aspectDecorator;
 
     private final ExceptionTracker _exceptionTracker;
 
-    public LoggingDecoratorImpl(@Builtin
-    ClassFactory classFactory,
-
-                                ExceptionTracker exceptionTracker)
+    public LoggingDecoratorImpl(AspectDecorator aspectDecorator, ExceptionTracker exceptionTracker)
     {
-        _classFactory = classFactory;
+        _aspectDecorator = aspectDecorator;
         _exceptionTracker = exceptionTracker;
     }
 
-    public <T> T build(Class<T> serviceInterface, T delegate, String serviceId, Logger logger)
+    public <T> T build(Class<T> serviceInterface, T delegate, String serviceId, final Logger logger)
     {
-        Class interceptorClass = createInterceptorClass(serviceInterface, serviceId);
-
-        ServiceLogger serviceLogger = new ServiceLogger(logger, _exceptionTracker);
-
-        Constructor cc = interceptorClass.getConstructors()[0];
-
-        Object interceptor = null;
-        Throwable fail = null;
+        final ServiceLogger serviceLogger = new ServiceLogger(logger, _exceptionTracker);
 
-        try
+        MethodAdvice advice = new MethodAdvice()
         {
-            interceptor = cc.newInstance(delegate, serviceLogger);
-        }
-        catch (InvocationTargetException ite)
-        {
-            fail = ite.getTargetException();
-        }
-        catch (Exception ex)
-        {
-            fail = ex;
-        }
+            public void advise(Invocation invocation)
+            {
+                boolean debug = logger.isDebugEnabled();
 
-        if (fail != null) throw new RuntimeException(fail);
+                if (debug) serviceLogger.entry(invocation);
 
-        return serviceInterface.cast(interceptor);
-    }
+                try
+                {
+                    invocation.proceed();
+                }
+                catch (RuntimeException ex)
+                {
+                    if (debug) serviceLogger.fail(invocation, ex);
 
-    private Class createInterceptorClass(Class serviceInterface, String serviceId)
-    {
-        ClassFab cf = _classFactory.newClass(serviceInterface);
+                    throw ex;
+                }
 
-        cf.addField("_delegate", Modifier.PRIVATE | Modifier.FINAL, serviceInterface);
-        cf.addField("_logger", Modifier.PRIVATE | Modifier.FINAL, ServiceLogger.class);
+                if (!debug) return;
 
-        cf.addConstructor(new Class[]
-                {serviceInterface, ServiceLogger.class}, null, "{ _delegate = $1; _logger = $2; }");
+                if (invocation.isFail())
+                {
+                    Exception thrown = invocation.getThrown(Exception.class);
 
-        addMethods(cf, serviceInterface, serviceId);
+                    serviceLogger.fail(invocation, thrown);
 
-        return cf.createClass();
-    }
-
-    private void addMethods(ClassFab cf, Class serviceInterface, String serviceId)
-    {
-        MethodIterator mi = new MethodIterator(serviceInterface);
+                    return;
+                }
 
-        while (mi.hasNext())
-            addMethod(cf, mi.next());
+                serviceLogger.exit(invocation);
+            }
+        };
 
-        if (!mi.getToString())
-            cf.addToString(ServiceMessages.loggingInterceptor(serviceId, serviceInterface));
+        return _aspectDecorator.build(serviceInterface, delegate, advice,
+                                      String.format("<Logging interceptor for %s(%s)>", serviceId,
+                                                    serviceInterface.getName()));
     }
 
-    private void addMethod(ClassFab cf, MethodSignature signature)
-    {
-        String name = '"' + signature.getName() + '"';
-        Class returnType = signature.getReturnType();
-        boolean isVoid = returnType.equals(void.class);
-
-        // We'll see how well Javassist handles void methods with this setup
-
-        BodyBuilder builder = new BodyBuilder();
-        builder.begin();
-        builder.addln("boolean debug = _logger.isDebugEnabled();");
-
-        builder.addln("if (debug)");
-        builder.addln("  _logger.entry(%s, $args);", name);
-
-        builder.addln("try");
-        builder.begin();
-
-        if (!isVoid) builder.add("%s result = ", toJavaClassName(returnType));
-
-        builder.addln("_delegate.%s($$);", signature.getName());
-
-        if (isVoid)
-        {
-            builder.addln("if (debug)");
-            builder.addln(format("  _logger.voidExit(%s);", name));
-            builder.addln("return;");
-        }
-        else
-        {
-            builder.addln("if (debug)");
-            builder.addln(format("  _logger.exit(%s, ($w)result);", name));
-            builder.addln("return result;");
-        }
-
-        builder.end(); // try
-
-        // Now, a catch for each declared exception (if any)
-
-        if (signature.getExceptionTypes() != null)
-            for (Class exceptionType : signature.getExceptionTypes())
-                addExceptionHandler(builder, name, exceptionType);
-
-        // And a catch for RuntimeException
-
-        addExceptionHandler(builder, name, RuntimeException.class);
-
-        builder.end();
-
-        cf.addMethod(Modifier.PUBLIC, signature, builder.toString());
-    }
-
-    private void addExceptionHandler(BodyBuilder builder, String quotedMethodName,
-                                     Class exceptionType)
-    {
-        builder.addln("catch (%s ex)", exceptionType.getName());
-        builder.begin();
-        builder.addln("if (debug)");
-        builder.addln("  _logger.fail(%s, ex);", quotedMethodName);
-        builder.addln("throw ex;");
-        builder.end();
-    }
 }

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/ServiceLogger.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/ServiceLogger.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/ServiceLogger.java (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/internal/services/ServiceLogger.java Sun Apr 20 11:56:22 2008
@@ -1,4 +1,4 @@
-// Copyright 2006, 2007 The Apache Software Foundation
+// Copyright 2006, 2007, 2008 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.
@@ -14,7 +14,7 @@
 
 package org.apache.tapestry.ioc.internal.services;
 
-import org.apache.tapestry.ioc.internal.util.Defense;
+import org.apache.tapestry.ioc.Invocation;
 import org.apache.tapestry.ioc.services.ExceptionTracker;
 import org.slf4j.Logger;
 
@@ -22,14 +22,12 @@
 import java.util.Iterator;
 
 /**
- * Used by {@link org.apache.tapestry.ioc.internal.services.LoggingDecoratorImpl} to delegate out
- * logging behavior to a seperate object (helps ensure no naming conflicts).
+ * Used by {@link org.apache.tapestry.ioc.internal.services.LoggingDecoratorImpl} to delegate out logging behavior to a
+ * seperate object (helps ensure no naming conflicts).
  */
 public final class ServiceLogger
 {
-    private final Logger _logger;
-
-    private final ExceptionTracker _exceptionTracker;
+    private static final int BUFFER_SIZE = 200;
 
     private static final String ENTER = "ENTER";
 
@@ -37,6 +35,10 @@
 
     private static final String FAIL = " FAIL";
 
+    private final Logger _logger;
+
+    private final ExceptionTracker _exceptionTracker;
+
     public ServiceLogger(Logger logger, ExceptionTracker exceptionTracker)
     {
         _logger = logger;
@@ -44,30 +46,21 @@
     }
 
     /**
-     * Returns true if the debugging is enabled for the underlying Log.
-     */
-    public boolean isDebugEnabled()
-    {
-        return _logger.isDebugEnabled();
-    }
-
-    /**
      * Invoked when a method is first entered
      *
-     * @param name      of the method
-     * @param arguments
+     * @param invocation identifies method invoked as well as parameters passed to method
      */
-    public void entry(String name, Object[] arguments)
+    public void entry(Invocation invocation)
     {
-        StringBuilder buffer = new StringBuilder();
+        StringBuilder buffer = new StringBuilder(BUFFER_SIZE);
 
-        buffer.append(format("[%s] %s(", ENTER, name));
+        buffer.append(format("[%s] %s(", ENTER, invocation.getMethodName()));
 
-        for (int i = 0; i < arguments.length; i++)
+        for (int i = 0; i < invocation.getParameterCount(); i++)
         {
             if (i > 0) buffer.append(", ");
 
-            convert(buffer, arguments[i]);
+            convert(buffer, invocation.getParameter(i));
         }
 
         buffer.append(")");
@@ -135,44 +128,38 @@
     }
 
     /**
-     * Invoked when a method returns a value
+     * Invoked when a method exits (possibly returning a value).
      *
-     * @param name   of the method
-     * @param result the return value for the method invocation
+     * @param invocation identifies method invocation and  result value
      */
-    public void exit(String name, Object result)
+    public void exit(Invocation invocation)
     {
-        Defense.notNull(name, "name");
-
-        StringBuilder buffer = new StringBuilder();
+        StringBuilder buffer = new StringBuilder(BUFFER_SIZE);
 
-        buffer.append(format("[%s] %s [", EXIT, name));
+        buffer.append(format("[%s] %s", EXIT, invocation.getMethodName()));
 
-        convert(buffer, result);
-
-        buffer.append(']');
+        if (invocation.getResultType() != void.class)
+        {
+            buffer.append(" [");
+            convert(buffer, invocation.getResult());
+            buffer.append(']');
+        }
 
         _logger.debug(buffer.toString());
     }
 
     /**
-     * Invoked when void method finishes succesfully.
-     */
-    public void voidExit(String name)
-    {
-        _logger.debug(format("[%s] %s", EXIT, name));
-    }
-
-    /**
      * Invoked when method invocation instead throws an exception.
+     *
+     * @param invocation identifies method invocation which failed
+     * @param t          exception throws by method invocation
      */
-    public void fail(String name, Throwable t)
+    public void fail(Invocation invocation, Throwable t)
     {
-        if (_logger.isDebugEnabled())
-        {
-            _logger.debug(
-                    format("[%s] %s -- %s", FAIL, name, t.getClass().getName()),
-                    _exceptionTracker.exceptionLogged(t) ? null : t);
-        }
+        _logger.debug(
+                format("[%s] %s -- %s", FAIL,
+                       invocation.getMethodName(),
+                       t.getClass().getName()),
+                _exceptionTracker.exceptionLogged(t) ? null : t);
     }
 }

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectDecorator.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectDecorator.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectDecorator.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectDecorator.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,50 @@
+// Copyright 2008 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.tapestry.ioc.services;
+
+import org.apache.tapestry.ioc.MethodAdvice;
+
+
+/**
+ * A decorator used to create an interceptor that delegates each method's invocation to an {@link
+ * org.apache.tapestry.ioc.MethodAdvice} for advise.
+ */
+public interface AspectDecorator
+{
+    /**
+     * Intercepts method invocations on the delegate and passes them through the provided aspect. Note that the advice
+     * <em>must</em> be thread-safe.
+     *
+     * @param serviceInterface defines the interface of the interceptor and delegate
+     * @param delegate         the object on which methods will be invoked
+     * @param advice           intercepts the method invocations on the delegate
+     * @param description      used as the toString() of the returned interceptor, unless toString() is part of the
+     *                         service interface
+     * @return the interceptor, wrapping the delegate with all the advice
+     */
+    <T> T build(Class<T> serviceInterface, T delegate, MethodAdvice advice, String description);
+
+    /**
+     * Creates a builder that can be used to create the builder.  This is used when only some of the methods of need to
+     * be advised, or when different methods need to recieve different advice.
+     *
+     * @param serviceInterface defines the interface of the interceptor and the delegate
+     * @param delegate         the object on which methods will be invokes
+     * @param description      used as the toString() of the interceptor unless toString() is part of the service
+     *                         interface
+     * @return a builder that can be used to generate the final interceptor
+     */
+    <T> AspectInterceptorBuilder<T> createBuilder(Class<T> serviceInterface, T delegate, String description);
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectInterceptorBuilder.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectInterceptorBuilder.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectInterceptorBuilder.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/AspectInterceptorBuilder.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,42 @@
+// Copyright 2008 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.tapestry.ioc.services;
+
+import org.apache.tapestry.ioc.MethodAdvice;
+
+import java.lang.reflect.Method;
+
+/**
+ * A builder may be obtained from the {@link org.apache.tapestry.ioc.services.AspectDecorator} and allows more
+ * controlled creation of the created interceptor; it allows different methods to be given different advice, and allows
+ * methods to be omitted (in which case the method invocation passes through without advice).
+ */
+public interface AspectInterceptorBuilder<T>
+{
+    /**
+     * Adds advice for a specific method of the aspect interceptor being constructed.
+     *
+     * @param method method (of the interface for which an interceptor is being constructed) to be adviced. A method may
+     *               only be advised <em>once</em> (for a single interceptor; it is valid to chain together a series of
+     *               interceptors).
+     * @param advice the advice for this particular method.   Advice must be threadsafe.
+     */
+    void adviseMethod(Method method, MethodAdvice advice);
+
+    /**
+     * Builds and returns the interceptor.  Any methods that have not been advised will become "pass thrus".
+     */
+    T build();
+}

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/TapestryIOCModule.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/TapestryIOCModule.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/TapestryIOCModule.java (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/main/java/org/apache/tapestry/ioc/services/TapestryIOCModule.java Sun Apr 20 11:56:22 2008
@@ -54,6 +54,7 @@
         binder.bind(Runnable.class, RegistryStartup.class).withId("RegistryStartup");
         binder.bind(MasterObjectProvider.class, MasterObjectProviderImpl.class);
         binder.bind(ClassNameLocator.class, ClassNameLocatorImpl.class);
+        binder.bind(AspectDecorator.class, AspectDecoratorImpl.class);
     }
 
     /**

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/site/apt/decorator.apt
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/site/apt/decorator.apt?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/site/apt/decorator.apt (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/site/apt/decorator.apt Sun Apr 20 11:56:22 2008
@@ -118,7 +118,7 @@
 public class MyAppModule
 {
   public static Indexer build(Class serviceInterface, Logger logger,
-    LoggingInterceptorFactory decorator)
+    LoggingDecorator decorator)
   {
     return decorator.build(serviceInterface, logger,  new IndexerImpl());
   } 
@@ -205,4 +205,110 @@
    and that is passed to the last decorator method. The interceptor created there
    is passed to the the next-to-last decorator method, and so forth.
 
-    
\ No newline at end of file
+Creating your own Decorators
+
+  Decorators are a limited form of Aspect Oriented Programming, so we have
+  borrowed some of that terminology here.
+
+  A decorator exists to create an <interceptor>.  The interceptor wraps around
+  the service (because these interceptors can get chained, we talk about the "delegate" and not the "service").
+
+  Each method of the interceptor will take <advice>.  Advice
+  is provided by a {{{../apidocs/org/apache/tapestry/ioc/MethodAdvice.html}MethodAdvice}} instance.
+  The sole method, <<<advise()>>>, recieves an
+   {{{../apidocs/org/apache/tapestry/ioc/Invocation.html}Invocation}}.
+  MethodAdvice gives you a chance to see what the method invocation <is>; you can query
+  the name of the method, and the types and values of the parameters.
+
+  The MethodAdvice can override the parameters if necessary, then invoke <<<proceed()>>>.  This
+  call invokes the corresponding method on the original object, the delegate.
+
+  If the method call throws a runtime exception, that exception is not caught.  Your method advice can
+  put a try ... catch block around the call to proceed() if interested in catching runtime exception.
+
+  Checked exceptions are not thrown (since they are not part of the proceed() method's signature).  Instead
+  the invocation's <<<isFail()>>> method will return true.  You can then retrieve the exception or override it.
+
+  In the normal success case, you can ask for the return value and even override it before
+  returning from the advise() method.
+
+  In other words, you have total control.  Your MethodAdvise can query or change parameters, decide whether
+  it proceed into the original code, it can intercept exceptions that are thrown and replace them, and can
+  query or even replace the return value.
+
+  The
+  {{{../apidocs/org/apache/tapestry/service/AspectDecorator.html}AspectDecorator}} service
+  is how you put your MethodAdvise into action.
+
+  By way of an example, we'll show the current implementation of the LoggingDecorator service:
+
++----+
+public class LoggingDecoratorImpl implements LoggingDecorator
+{
+    private final AspectDecorator _aspectDecorator;
+
+    private final ExceptionTracker _exceptionTracker;
+
+    public LoggingDecoratorImpl(AspectDecorator aspectDecorator, ExceptionTracker exceptionTracker)
+    {
+        _aspectDecorator = aspectDecorator;
+        _exceptionTracker = exceptionTracker;
+    }
+
+    public <T> T build(Class<T> serviceInterface, T delegate, String serviceId, final Logger logger)
+    {
+        final ServiceLogger serviceLogger = new ServiceLogger(logger, _exceptionTracker);
+
+        MethodAdvice advice = new MethodAdvice()
+        {
+            public void advise(Invocation invocation)
+            {
+                boolean debug = logger.isDebugEnabled();
+
+                if (debug) serviceLogger.entry(invocation);
+
+                try
+                {
+                    invocation.proceed();
+                }
+                catch (RuntimeException ex)
+                {
+                    if (debug) serviceLogger.fail(invocation, ex);
+
+                    throw ex;
+                }
+
+                if (!debug) return;
+
+                if (invocation.isFail())
+                {
+                    Exception thrown = invocation.getThrown(Exception.class);
+
+                    serviceLogger.fail(invocation, thrown);
+
+                    return;
+                }
+
+                serviceLogger.exit(invocation);
+            }
+        };
+
+        return _aspectDecorator.build(serviceInterface, delegate, advice,
+                                      String.format("<Logging interceptor for %s(%s)>", serviceId,
+                                                    serviceInterface.getName()));
+    }
+}
++---+
+
+  Most of the logging logic occurs inside the ServiceLogger object, the MethodAdvise exists to call the right methods at
+  the right time.  A Logger doesn't <change> parameter values (or thrown exceptions, or the result), it just
+  captures and logs the data.
+
+  Notice that for runtime exceptions, we catch the exception, log it, and rethrow it.
+
+  For checked exceptions, we use isFail() and getThrown().
+
+  The AspectDecorator service can also be used in more complicated ways: it is possible to
+  only advise some of the methods and not others, or use different advise for different methods.  Check the
+  JavaDoc for more details.
+

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AbstractInvocationTest.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AbstractInvocationTest.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AbstractInvocationTest.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AbstractInvocationTest.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,136 @@
+// Copyright 2008 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.tapestry.ioc.internal.services;
+
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.test.TestBase;
+import org.testng.annotations.Test;
+
+import java.lang.reflect.Method;
+import java.sql.SQLException;
+
+
+public class AbstractInvocationTest extends TestBase
+{
+    class TestInvocation extends AbstractInvocation
+    {
+        protected TestInvocation(Method method)
+        {
+            super(method);
+        }
+
+        public Object getParameter(int index)
+        {
+            return null;
+        }
+
+        public void override(int index, Object newParameter)
+        {
+        }
+
+        public void proceed()
+        {
+        }
+    }
+
+    interface Subject
+    {
+        void go() throws SQLException;
+
+        int count() throws SQLException;
+
+        void execute(Runnable runnable);
+    }
+
+    @Test
+    public void to_string() throws Exception
+    {
+        Invocation iv = new TestInvocation(Runnable.class.getMethod("run"));
+
+        assertEquals(iv.toString(), "Invocation[public abstract void java.lang.Runnable.run()]");
+    }
+
+    @Test
+    public void override_exception() throws Exception
+    {
+        SQLException se = new SQLException();
+
+        Invocation iv = new TestInvocation(Subject.class.getMethod("go"));
+
+        iv.overrideThrown(se);
+
+        assertTrue(iv.isFail());
+        assertSame(iv.getThrown(Exception.class), se);
+    }
+
+    @Test
+    public void get_thrown_returns_null_if_not_a_match() throws Exception
+    {
+        SQLException se = new SQLException();
+
+        Invocation iv = new TestInvocation(Subject.class.getMethod("go"));
+
+        iv.overrideThrown(se);
+
+        assertNull(iv.getThrown(RuntimeException.class));
+    }
+
+    @Test
+    public void override_result_clears_exception() throws Exception
+    {
+        SQLException se = new SQLException();
+        Integer override = new Integer(23);
+
+        Invocation iv = new TestInvocation(Subject.class.getMethod("count"));
+
+        iv.overrideThrown(se);
+
+        assertTrue(iv.isFail());
+
+        iv.overrideResult(override);
+        assertFalse(iv.isFail());
+        assertSame(iv.getResult(), override);
+    }
+
+    @Test
+    public void invalid_exception_for_override() throws Exception
+    {
+        SQLException se = new SQLException();
+
+        Invocation iv = new TestInvocation(Runnable.class.getMethod("run"));
+
+        try
+        {
+            iv.overrideThrown(se);
+            unreachable();
+        }
+        catch (IllegalArgumentException ex)
+        {
+            assertEquals(ex.getMessage(),
+                         "Exception java.sql.SQLException is not a declared exception of method public abstract void java.lang.Runnable.run().");
+        }
+    }
+
+    @Test
+    public void get_parameter_type() throws Exception
+    {
+        Invocation iv = new TestInvocation(Subject.class.getMethod("execute", Runnable.class));
+
+        assertEquals(iv.getParameterCount(), 1);
+        assertSame(iv.getParameterType(0), Runnable.class);
+
+    }
+
+}

Added: tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImplTest.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImplTest.java?rev=649954&view=auto
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImplTest.java (added)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/AspectInterceptorBuilderImplTest.java Sun Apr 20 11:56:22 2008
@@ -0,0 +1,147 @@
+// Copyright 2008 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.tapestry.ioc.internal.services;
+
+import org.apache.tapestry.ioc.Invocation;
+import org.apache.tapestry.ioc.MethodAdvice;
+import org.apache.tapestry.ioc.internal.IOCInternalTestCase;
+import org.apache.tapestry.ioc.services.AspectDecorator;
+import org.apache.tapestry.ioc.services.AspectInterceptorBuilder;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+/**
+ * Tests a few edge and error cases not covered by {@link org.apache.tapestry.ioc.internal.services.LoggingDecoratorImplTest}.
+ */
+public class AspectInterceptorBuilderImplTest extends IOCInternalTestCase
+{
+    private AspectDecorator _decorator;
+
+    @BeforeClass
+    public void setup()
+    {
+        _decorator = getService(AspectDecorator.class);
+    }
+
+    public interface Subject
+    {
+        void advised();
+
+        void notAdvised();
+    }
+
+    @Test
+    public void some_methods_not_intercepted() throws Exception
+    {
+        Subject delegate = mockSubject();
+
+        MethodAdvice advice = new MethodAdvice()
+        {
+            public void advise(Invocation invocation)
+            {
+                assertEquals(invocation.getMethodName(), "advised");
+
+                invocation.proceed();
+            }
+        };
+
+        delegate.advised();
+        delegate.notAdvised();
+
+        replay();
+
+        AspectInterceptorBuilder<Subject> builder = _decorator.createBuilder(Subject.class, delegate, "<Subject>");
+
+        builder.adviseMethod(Subject.class.getMethod("advised"), advice);
+
+        Subject interceptor = builder.build();
+
+        interceptor.advised();
+        interceptor.notAdvised();
+
+        verify();
+    }
+
+    @Test
+    public void method_not_in_service_interface() throws Exception
+    {
+        Subject delegate = mockSubject();
+
+        MethodAdvice advice = mockAdvice();
+
+        replay();
+
+        AspectInterceptorBuilder<Subject> builder = _decorator.createBuilder(Subject.class, delegate, "<Subject>");
+
+        // This method doesn't belong.
+
+        try
+        {
+            builder.adviseMethod(Runnable.class.getMethod("run"), advice);
+
+            unreachable();
+        }
+        catch (IllegalArgumentException ex)
+        {
+            assertEquals(ex.getMessage(),
+                         "Method public abstract void java.lang.Runnable.run() is not defined for interface interface org.apache.tapestry.ioc.internal.services.AspectInterceptorBuilderImplTest$Subject.");
+        }
+
+
+        verify();
+    }
+
+    @Test
+    public void method_with_duplicate_advice() throws Exception
+    {
+        Subject delegate = mockSubject();
+
+        MethodAdvice advice = mockAdvice();
+
+        replay();
+
+        AspectInterceptorBuilder<Subject> builder = _decorator.createBuilder(Subject.class, delegate, "<Subject>");
+
+
+        builder.adviseMethod(Subject.class.getMethod("advised"), advice);
+
+        try
+        {
+            // Second is failure.
+
+            builder.adviseMethod(Subject.class.getMethod("advised"), advice);
+
+            unreachable();
+        }
+        catch (IllegalArgumentException ex)
+        {
+            assertEquals(ex.getMessage(),
+                         "Method public abstract void org.apache.tapestry.ioc.internal.services.AspectInterceptorBuilderImplTest$Subject.advised() has already been advised.");
+        }
+
+        verify();
+    }
+
+    protected final MethodAdvice mockAdvice()
+    {
+        return newMock(MethodAdvice.class);
+    }
+
+    protected final Subject mockSubject()
+    {
+        return newMock(Subject.class);
+    }
+
+}

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImplTest.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImplTest.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImplTest.java (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/internal/services/LoggingDecoratorImplTest.java Sun Apr 20 11:56:22 2008
@@ -1,4 +1,4 @@
-// Copyright 2006, 2007 The Apache Software Foundation
+// Copyright 2006, 2007, 2008 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.
@@ -14,20 +14,32 @@
 
 package org.apache.tapestry.ioc.internal.services;
 
+import org.apache.tapestry.ioc.internal.IOCInternalTestCase;
+import org.apache.tapestry.ioc.services.AspectDecorator;
 import org.apache.tapestry.ioc.services.LoggingDecorator;
-import org.apache.tapestry.ioc.test.IOCTestCase;
 import org.slf4j.Logger;
 import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
 import org.testng.annotations.Test;
 import org.xml.sax.SAXParseException;
 
 /**
- * Use the LoggingDecorator in a number of ways to verify its behavior. In some ways we are testing
- * the code dynamically generated by the LoggingDecorator as much as we are testing the decorator
- * itself -- one proves the other.
+ * Use the LoggingDecorator in a number of ways to verify its behavior. In some ways we are testing the code dynamically
+ * generated by the LoggingDecorator as much as we are testing the decorator itself -- one proves the other.
+ * <p/>
+ * And now this test is used to integration test {@link org.apache.tapestry.ioc.internal.services.AspectDecoratorImpl}
+ * as well.
  */
-public class LoggingDecoratorImplTest extends IOCTestCase
+public class LoggingDecoratorImplTest extends IOCInternalTestCase
 {
+    private AspectDecorator _aspectDecorator;
+
+    @BeforeClass
+    public void setup()
+    {
+        _aspectDecorator = getService(AspectDecorator.class);
+    }
+
     public interface UpcaseService
     {
         String upcase(String input);
@@ -77,7 +89,7 @@
 
     private LoggingDecoratorImpl newLoggingDecorator()
     {
-        return new LoggingDecoratorImpl(new ClassFactoryImpl(), new ExceptionTrackerImpl());
+        return new LoggingDecoratorImpl(_aspectDecorator, new ExceptionTrackerImpl());
     }
 
     @Test
@@ -93,8 +105,6 @@
         delegate.run();
         setThrowable(t);
 
-        train_isDebugEnabled(logger, true);
-
         logger.debug("[ FAIL] run -- " + t.getClass().getName(), t);
 
         replay();
@@ -127,8 +137,6 @@
 
         delegate.parse();
         setThrowable(t);
-
-        train_isDebugEnabled(logger, true);
 
         logger.debug("[ FAIL] parse -- " + t.getClass().getName(), t);
 

Modified: tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/util/TimeIntervalTest.java
URL: http://svn.apache.org/viewvc/tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/util/TimeIntervalTest.java?rev=649954&r1=649953&r2=649954&view=diff
==============================================================================
--- tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/util/TimeIntervalTest.java (original)
+++ tapestry/tapestry5/trunk/tapestry-ioc/src/test/java/org/apache/tapestry/ioc/util/TimeIntervalTest.java Sun Apr 20 11:56:22 2008
@@ -1,4 +1,4 @@
-// Copyright 2007 The Apache Software Foundation
+// Copyright 2007, 2008 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.
@@ -42,7 +42,7 @@
         catch (RuntimeException ex)
         {
             assertEquals(ex.getMessage(),
-                         "Unknown time interval unit 'mz' (in '30s 500mz').  Defined units: d, h, m, ms, s.");
+                         "Unknown time interval unit 'mz' (in '30s 500mz').  Defined units: d, h, m, ms, s, y.");
         }
     }
 
@@ -83,17 +83,17 @@
     @DataProvider(name = "mix_of_units_data")
     public Object[][] mix_of_units_data()
     {
-        return new Object[][]{{"54321", 54321},
+        return new Object[][] { { "54321", 54321 },
 
-                              {"30s", 30 * 1000},
+                { "30s", 30 * 1000 },
 
-                              {"1h 30m", 90 * 60 * 1000},
+                { "1h 30m", 90 * 60 * 1000 },
 
-                              {"2d", 2 * 24 * 60 * 60 * 1000},
+                { "2d", 2 * 24 * 60 * 60 * 1000 },
 
-                              {"2m", 2 * 60 * 1000},
+                { "2m", 2 * 60 * 1000 },
 
-                              {"23ms", 23}
+                { "23ms", 23 }
 
         };
     }