You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@isis.apache.org by da...@apache.org on 2016/04/23 16:55:23 UTC

[2/5] isis git commit: ISIS-1390: default CommandServiceDefault. Also simplifying the relationship between IsisTransaction and Command, and how the Command object is initialized.

ISIS-1390: default CommandServiceDefault.  Also simplifying the relationship between IsisTransaction and Command, and how the Command object is initialized.

Also:
... deprecating CommandService#startTransaction(...), make it the responsibility of IsisTransactionManager to populate the transactionId of the Command in all cases.
... simplifying logic in IsisTransaction; the Command is only just created, so will never have a transactionId to reuse.  The design is now simply for the IsisTransactionManager to create a new UUID and set on both the Command and also pass into IsisTransaction.
... extend IsisTranscationManager's executeWithTransaction to allow an existing Command to be passed in; leverage this from BackgroundCommandExecution.

Also:
... use ClockService to obtain the time, rather than Clock singleton (for Command#setTimestamp, also Command#setStartedAt,
... make lookup of mandatory services fail-fast if not present (PropertySetterFacet, ActionInvocationFacet, IsisTransactionManager, IsisTransaction)
... minor alterations to visibility of DomainObjectJdoAbstract properties in tables.


Project: http://git-wip-us.apache.org/repos/asf/isis/repo
Commit: http://git-wip-us.apache.org/repos/asf/isis/commit/98a0c2f8
Tree: http://git-wip-us.apache.org/repos/asf/isis/tree/98a0c2f8
Diff: http://git-wip-us.apache.org/repos/asf/isis/diff/98a0c2f8

Branch: refs/heads/ISIS-1291
Commit: 98a0c2f8032f49178e9a048d4139e4d0af66c594
Parents: 83d1bc0
Author: Dan Haywood <da...@haywood-associates.co.uk>
Authored: Sat Apr 23 12:47:21 2016 +0100
Committer: Dan Haywood <da...@haywood-associates.co.uk>
Committed: Sat Apr 23 12:47:21 2016 +0100

----------------------------------------------------------------------
 .../isis/applib/services/HasTransactionId.java  |   4 +-
 .../isis/applib/services/command/Command3.java  |   2 +-
 .../services/command/spi/CommandService.java    |  14 +-
 .../services/eventbus/ActionDomainEvent.java    |   4 +-
 .../integtestsupport/IsisSystemForTest.java     |   7 +-
 ...onInvocationFacetForDomainEventAbstract.java |  39 ++--
 .../PropertySetterFacetViaModifyMethod.java     |  29 ++-
 .../background/BackgroundCommandExecution.java  |  35 ++--
 .../services/command/CommandServiceDefault.java |  71 +++++++
 .../system/transaction/IsisTransaction.java     | 108 +++++-----
 .../transaction/IsisTransactionManager.java     | 202 ++++++++-----------
 11 files changed, 286 insertions(+), 229 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/applib/src/main/java/org/apache/isis/applib/services/HasTransactionId.java
----------------------------------------------------------------------
diff --git a/core/applib/src/main/java/org/apache/isis/applib/services/HasTransactionId.java b/core/applib/src/main/java/org/apache/isis/applib/services/HasTransactionId.java
index 424715b..69b33d5 100644
--- a/core/applib/src/main/java/org/apache/isis/applib/services/HasTransactionId.java
+++ b/core/applib/src/main/java/org/apache/isis/applib/services/HasTransactionId.java
@@ -38,9 +38,9 @@ public interface HasTransactionId {
      * Note that this is the same as the Isis transaction guid as found in the JDO applib's
      * <tt>PublishedEvent</tt>.
      */
-    public UUID getTransactionId();
+    UUID getTransactionId();
 
-    public void setTransactionId(final UUID transactionId);
+    void setTransactionId(final UUID transactionId);
 
     
 

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/applib/src/main/java/org/apache/isis/applib/services/command/Command3.java
----------------------------------------------------------------------
diff --git a/core/applib/src/main/java/org/apache/isis/applib/services/command/Command3.java b/core/applib/src/main/java/org/apache/isis/applib/services/command/Command3.java
index 5b65f5b..a4e1648 100644
--- a/core/applib/src/main/java/org/apache/isis/applib/services/command/Command3.java
+++ b/core/applib/src/main/java/org/apache/isis/applib/services/command/Command3.java
@@ -49,7 +49,7 @@ public interface Command3 extends Command2 {
     ActionDomainEvent<?> popActionDomainEvent();
 
     /**
-     * @deprecated - use {@link Interaction#getDomainEvents()} and {@link Interaction#clearDomainEvents()} instead.
+     * @deprecated - use {@link Interaction#getExecutionGraphs()} and {@link Interaction#clearDomainEvents()} instead.
      */
     @Deprecated
     @Programmatic

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/applib/src/main/java/org/apache/isis/applib/services/command/spi/CommandService.java
----------------------------------------------------------------------
diff --git a/core/applib/src/main/java/org/apache/isis/applib/services/command/spi/CommandService.java b/core/applib/src/main/java/org/apache/isis/applib/services/command/spi/CommandService.java
index 7c7bb7a..ab33170 100644
--- a/core/applib/src/main/java/org/apache/isis/applib/services/command/spi/CommandService.java
+++ b/core/applib/src/main/java/org/apache/isis/applib/services/command/spi/CommandService.java
@@ -31,19 +31,17 @@ import org.apache.isis.applib.services.command.Command;
  */
 public interface CommandService {
 
+    /**
+     * Simply instantiate the appropriate instance of the {@link Command}.  Its members will be populated automatically
+     * by the framework.
+     */
     @Programmatic
     Command create();
     
     /**
-     * Although the transactionId is also provided in the
-     * {@link #complete(Command)} callback, it is passed in here as well
-     * so that an implementation can ensure that the {@link Command} is fully populated in order
-     * to persist if required.
-     * 
-     * <p>
-     * One case where this may be supported (for example, by the <tt>CommandServiceJdo</tt> implementation)
-     * is to flush still-running {@link Command}s to the database on-demand.
+     * @deprecated - the framework automatically populates the {@link Command}'s {@link Command#getTimestamp()}, {@link Command#getUser()}  and {@link Command#getTransactionId()}, so there is no need for the service implementation to initialize any of these.  In particular, the {@link Command} will already have been initialized with the provided <tt>transactionId</tt> argument.
      */
+    @Deprecated
     @Programmatic
     void startTransaction(final Command command, final UUID transactionId);
     

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/applib/src/main/java/org/apache/isis/applib/services/eventbus/ActionDomainEvent.java
----------------------------------------------------------------------
diff --git a/core/applib/src/main/java/org/apache/isis/applib/services/eventbus/ActionDomainEvent.java b/core/applib/src/main/java/org/apache/isis/applib/services/eventbus/ActionDomainEvent.java
index d2d0f1c..d15b214 100644
--- a/core/applib/src/main/java/org/apache/isis/applib/services/eventbus/ActionDomainEvent.java
+++ b/core/applib/src/main/java/org/apache/isis/applib/services/eventbus/ActionDomainEvent.java
@@ -131,12 +131,12 @@ public abstract class ActionDomainEvent<S> extends AbstractInteractionEvent<S> {
      * The {@link org.apache.isis.applib.services.command.Command} for this action.
      *
      * <p>
-     * Set when in {@link org.apache.isis.applib.services.eventbus.AbstractInteractionEvent.Phase#EXECUTING} and {@link org.apache.isis.applib.services.eventbus.AbstractInteractionEvent.Phase#EXECUTED}, but not for earlier phases.
+     * Set when in {@link org.apache.isis.applib.services.eventbus.AbstractDomainEvent.Phase#EXECUTING} and {@link org.apache.isis.applib.services.eventbus.AbstractDomainEvent.Phase#EXECUTED}, but not for earlier phases.
      *
      * <p>
      * The command is set by the framework based on the configured
      * {@link org.apache.isis.applib.services.command.CommandContext}) service).  Ths command may or may not be
-     * persisted, depending on the which implementation of
+     * persisted, depending on which implementation of
      * {@link org.apache.isis.applib.services.command.spi.CommandService} service is configured.
      */
     public Command getCommand() {

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/integtestsupport/src/main/java/org/apache/isis/core/integtestsupport/IsisSystemForTest.java
----------------------------------------------------------------------
diff --git a/core/integtestsupport/src/main/java/org/apache/isis/core/integtestsupport/IsisSystemForTest.java b/core/integtestsupport/src/main/java/org/apache/isis/core/integtestsupport/IsisSystemForTest.java
index 16b4f84..52c947f 100644
--- a/core/integtestsupport/src/main/java/org/apache/isis/core/integtestsupport/IsisSystemForTest.java
+++ b/core/integtestsupport/src/main/java/org/apache/isis/core/integtestsupport/IsisSystemForTest.java
@@ -698,11 +698,8 @@ public class IsisSystemForTest implements org.junit.rules.TestRule, DomainServic
 
         // specify that this command (if any) is being executed by a 'USER'
         final CommandContext commandContext = getService(CommandContext.class);
-        Command command;
-        if (commandContext != null) {
-            command = commandContext.getCommand();
-            command.setExecutor(Command.Executor.USER);
-        }
+        Command command = commandContext.getCommand();
+        command.setExecutor(Command.Executor.USER);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/actions/action/invocation/ActionInvocationFacetForDomainEventAbstract.java
----------------------------------------------------------------------
diff --git a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/actions/action/invocation/ActionInvocationFacetForDomainEventAbstract.java b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/actions/action/invocation/ActionInvocationFacetForDomainEventAbstract.java
index bac5458..8f868a3 100644
--- a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/actions/action/invocation/ActionInvocationFacetForDomainEventAbstract.java
+++ b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/actions/action/invocation/ActionInvocationFacetForDomainEventAbstract.java
@@ -34,9 +34,9 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.isis.applib.NonRecoverableException;
 import org.apache.isis.applib.RecoverableException;
-import org.apache.isis.applib.clock.Clock;
 import org.apache.isis.applib.services.bookmark.Bookmark;
 import org.apache.isis.applib.services.bookmark.BookmarkService;
+import org.apache.isis.applib.services.clock.ClockService;
 import org.apache.isis.applib.services.command.Command;
 import org.apache.isis.applib.services.command.CommandContext;
 import org.apache.isis.applib.services.command.spi.CommandService;
@@ -239,7 +239,7 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
 
             owningAction.setupCommand(targetAdapter, arguments);
 
-            ObjectAdapter resultAdapter = invokeThruCommand(owningAction, targetAdapter, arguments, command);
+            ObjectAdapter resultAdapter = invokeThruCommand(owningAction, targetAdapter, arguments);
 
             return InvocationResult.forActionThatReturned(resultAdapter);
 
@@ -296,8 +296,13 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
     protected ObjectAdapter invokeThruCommand(
             final ObjectAction owningAction,
             final ObjectAdapter targetAdapter,
-            final ObjectAdapter[] arguments, final Command command)
+            final ObjectAdapter[] arguments)
             throws IllegalAccessException, InvocationTargetException {
+
+        final CommandContext commandContext = getCommandContext();
+        final Command command = commandContext.getCommand();
+
+
         final ObjectAdapter resultAdapter;
         if( command.getExecutor() == Command.Executor.USER &&
                 command.getExecuteIn() == org.apache.isis.applib.annotation.Command.ExecuteIn.BACKGROUND) {
@@ -317,7 +322,7 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
         } else {
 
             // otherwise, go ahead and execute action in the 'foreground'
-            command.setStartedAt(Clock.getTimeAsJavaSqlTimestamp());
+            command.setStartedAt(getClockService().nowAsJavaSqlTimestamp());
 
             final Object resultPojo = invokeMethodElseFromCache(targetAdapter, arguments);
 
@@ -427,19 +432,19 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
     }
 
     private MetaModelService2 getMetaModelService() {
-        return lookupService(MetaModelService2.class);
+        return lookupServiceIfAny(MetaModelService2.class);
     }
 
     private TransactionService getTransactionService() {
-        return lookupService(TransactionService.class);
+        return lookupServiceIfAny(TransactionService.class);
     }
 
     private BookmarkService getBookmarkService() {
-        return lookupService(BookmarkService.class);
+        return lookupServiceIfAny(BookmarkService.class);
     }
 
     private RepositoryService getRepositoryService() {
-        return lookupService(RepositoryService.class);
+        return lookupServiceIfAny(RepositoryService.class);
     }
 
     protected void captureCurrentInvocationForPublishing(
@@ -531,11 +536,7 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
     // /////////////////////////////////////////////////////////
 
     private CommandContext getCommandContext() {
-        CommandContext commandContext = lookupService(CommandContext.class);
-        if (commandContext == null) {
-            throw new IllegalStateException("The CommandContext service is not registered!");
-        }
-        return commandContext;
+        return lookupService(CommandContext.class);
     }
 
     private QueryResultsCache getQueryResultsCache() {
@@ -546,7 +547,19 @@ public abstract class ActionInvocationFacetForDomainEventAbstract
         return lookupService(CommandService.class);
     }
 
+    private ClockService getClockService() {
+        return lookupService(ClockService.class);
+    }
+
     private <T> T lookupService(final Class<T> serviceClass) {
+        T service = lookupServiceIfAny(serviceClass);
+        if(service == null) {
+            throw new IllegalStateException("The '" + serviceClass.getName() + "' service is not registered!");
+        }
+        return service;
+    }
+
+    private <T> T lookupServiceIfAny(final Class<T> serviceClass) {
         return getServicesInjector().lookupService(serviceClass);
     }
 

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/properties/update/modify/PropertySetterFacetViaModifyMethod.java
----------------------------------------------------------------------
diff --git a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/properties/update/modify/PropertySetterFacetViaModifyMethod.java b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/properties/update/modify/PropertySetterFacetViaModifyMethod.java
index 32865c5..a220e64 100644
--- a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/properties/update/modify/PropertySetterFacetViaModifyMethod.java
+++ b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/facets/properties/update/modify/PropertySetterFacetViaModifyMethod.java
@@ -22,7 +22,8 @@ package org.apache.isis.core.metamodel.facets.properties.update.modify;
 import java.lang.reflect.Method;
 import java.util.Collections;
 import java.util.List;
-import org.apache.isis.applib.clock.Clock;
+
+import org.apache.isis.applib.services.clock.ClockService;
 import org.apache.isis.applib.services.command.Command;
 import org.apache.isis.applib.services.command.CommandContext;
 import org.apache.isis.core.metamodel.adapter.ObjectAdapter;
@@ -63,7 +64,7 @@ public class PropertySetterFacetViaModifyMethod extends PropertySetterFacetAbstr
             final ObjectAdapter valueAdapter,
             final InteractionInitiatedBy interactionInitiatedBy) {
 
-        final CommandContext commandContext = getServicesInjector().lookupService(CommandContext.class);
+        final CommandContext commandContext = getCommandContext();
         final Command command = commandContext.getCommand();
 
         // cf similar code in ActionInvocationFacetForDomainEventFacet
@@ -77,7 +78,7 @@ public class PropertySetterFacetViaModifyMethod extends PropertySetterFacetAbstr
         command.setExecuteIn(org.apache.isis.applib.annotation.Command.ExecuteIn.FOREGROUND);
         command.setPersistence(org.apache.isis.applib.annotation.Command.Persistence.IF_HINTED);
 
-        command.setStartedAt(Clock.getTimeAsJavaSqlTimestamp());
+        command.setStartedAt(getClockService().nowAsJavaSqlTimestamp());
 
         ObjectAdapter.InvokeUtils.invoke(method, targetAdapter, valueAdapter);
     }
@@ -92,16 +93,24 @@ public class PropertySetterFacetViaModifyMethod extends PropertySetterFacetAbstr
         return servicesInjector;
     }
 
-    private <T> T lookupService(final Class<T> serviceClass) {
-        return getServicesInjector().lookupService(serviceClass);
+    private CommandContext getCommandContext() {
+        return lookupService(CommandContext.class);
+    }
+
+    private ClockService getClockService() {
+        return lookupService(ClockService.class);
     }
 
-    protected CommandContext getCommandContext() {
-        CommandContext commandContext = lookupService(CommandContext.class);
-        if (commandContext == null) {
-            throw new IllegalStateException("The CommandContext service is not registered!");
+    private <T> T lookupService(final Class<T> serviceClass) {
+        T service = lookupServiceIfAny(serviceClass);
+        if(service == null) {
+            throw new IllegalStateException("The '" + serviceClass.getName() + "' service is not registered!");
         }
-        return commandContext;
+        return service;
+    }
+    private <T> T lookupServiceIfAny(final Class<T> serviceClass) {
+        return getServicesInjector().lookupService(serviceClass);
     }
 
+
 }

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/runtime/src/main/java/org/apache/isis/core/runtime/services/background/BackgroundCommandExecution.java
----------------------------------------------------------------------
diff --git a/core/runtime/src/main/java/org/apache/isis/core/runtime/services/background/BackgroundCommandExecution.java b/core/runtime/src/main/java/org/apache/isis/core/runtime/services/background/BackgroundCommandExecution.java
index fbed501..84b66a0 100644
--- a/core/runtime/src/main/java/org/apache/isis/core/runtime/services/background/BackgroundCommandExecution.java
+++ b/core/runtime/src/main/java/org/apache/isis/core/runtime/services/background/BackgroundCommandExecution.java
@@ -23,10 +23,10 @@ import com.google.common.base.Throwables;
 import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 
-import org.apache.isis.applib.clock.Clock;
 import org.apache.isis.applib.services.background.ActionInvocationMemento;
 import org.apache.isis.applib.services.bookmark.Bookmark;
 import org.apache.isis.applib.services.bookmark.BookmarkService;
+import org.apache.isis.applib.services.clock.ClockService;
 import org.apache.isis.applib.services.command.Command;
 import org.apache.isis.applib.services.command.Command.Executor;
 import org.apache.isis.applib.services.command.CommandContext;
@@ -72,16 +72,16 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
 
         final PersistenceSession persistenceSession = getPersistenceSession();
         final IsisTransactionManager transactionManager = getTransactionManager(persistenceSession);
-        final List<Command> commands = Lists.newArrayList();
+        final List<Command> backgroundCommands = Lists.newArrayList();
         transactionManager.executeWithinTransaction(new TransactionalClosure() {
             @Override
             public void execute() {
-                commands.addAll(findBackgroundCommandsToExecute());
+                backgroundCommands.addAll(findBackgroundCommandsToExecute());
             }
         });
 
-        for (final Command command : commands) {
-            execute(transactionManager, command);
+        for (final Command backgroundCommand : backgroundCommands) {
+            execute(transactionManager, backgroundCommand);
         }
     }
 
@@ -93,17 +93,17 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
     // //////////////////////////////////////
 
     
-    private void execute(final IsisTransactionManager transactionManager, final Command command) {
-        transactionManager.executeWithinTransaction(new TransactionalClosure() {
+    private void execute(final IsisTransactionManager transactionManager, final Command backgroundCommand) {
+        transactionManager.executeWithinTransaction(
+                backgroundCommand,
+                new TransactionalClosure() {
             @Override
             public void execute() {
 
-                commandContext.setCommand(command);
-                final String memento = command.getMemento();
+                final String memento = backgroundCommand.getMemento();
 
                 try {
-                    command.setStartedAt(Clock.getTimeAsJavaSqlTimestamp());
-                    command.setExecutor(Executor.BACKGROUND);
+                    backgroundCommand.setExecutor(Executor.BACKGROUND);
 
                     final boolean legacy = memento.startsWith("<memento");
                     if(legacy) {
@@ -120,7 +120,7 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
 
                         final ObjectAction objectAction = findAction(specification, actionId);
                         if(objectAction == null) {
-                            throw new Exception("Unknown action '" + actionId + "'");
+                            throw new Exception(String.format("Unknown action '%s'", actionId));
                         }
 
                         final ObjectAdapter[] argAdapters = argAdaptersFor(aim);
@@ -128,7 +128,7 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
                                 targetAdapter, argAdapters, InteractionInitiatedBy.FRAMEWORK);
                         if(resultAdapter != null) {
                             Bookmark resultBookmark = CommandUtil.bookmarkFor(resultAdapter);
-                            command.setResult(resultBookmark);
+                            backgroundCommand.setResult(resultBookmark);
                         }
 
                     } else {
@@ -157,7 +157,7 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
                             // responsibiity of auditing/profiling
                             if(resultAdapter != null) {
                                 Bookmark resultBookmark = CommandUtil.bookmarkFor(resultAdapter);
-                                command.setResult(resultBookmark);
+                                backgroundCommand.setResult(resultBookmark);
                             }
                         }
                     }
@@ -166,11 +166,11 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
                     // this doesn't really make sense if >1 action
                     // in any case, the capturing of the action interaction should be the
                     // responsibiity of auditing/profiling
-                    command.setException(Throwables.getStackTraceAsString(e));
+                    backgroundCommand.setException(Throwables.getStackTraceAsString(e));
                 } finally {
                     // decided to keep this, even though really this
                     // should be the responsibility of auditing/profiling
-                    command.setCompletedAt(Clock.getTimeAsJavaSqlTimestamp());
+                    backgroundCommand.setCompletedAt(clockService.nowAsJavaSqlTimestamp());
                 }
             }
 
@@ -252,4 +252,7 @@ public abstract class BackgroundCommandExecution extends AbstractIsisSessionTemp
 
     @javax.inject.Inject
     private CommandContext commandContext;
+
+    @javax.inject.Inject
+    private ClockService clockService;
 }

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/runtime/src/main/java/org/apache/isis/core/runtime/services/command/CommandServiceDefault.java
----------------------------------------------------------------------
diff --git a/core/runtime/src/main/java/org/apache/isis/core/runtime/services/command/CommandServiceDefault.java b/core/runtime/src/main/java/org/apache/isis/core/runtime/services/command/CommandServiceDefault.java
new file mode 100644
index 0000000..97439aa
--- /dev/null
+++ b/core/runtime/src/main/java/org/apache/isis/core/runtime/services/command/CommandServiceDefault.java
@@ -0,0 +1,71 @@
+/**
+ *  Licensed to the Apache Software Foundation (ASF) under one or more
+ *  contributor license agreements.  See the NOTICE file distributed with
+ *  this work for additional information regarding copyright ownership.
+ *  The ASF licenses this file to You under the Apache License, Version 2.0
+ *  (the "License"); you may not use this file except in compliance with
+ *  the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+package org.apache.isis.core.runtime.services.command;
+
+import java.util.UUID;
+
+import javax.inject.Inject;
+
+import org.apache.isis.applib.annotation.DomainService;
+import org.apache.isis.applib.annotation.NatureOfService;
+import org.apache.isis.applib.annotation.Programmatic;
+import org.apache.isis.applib.services.clock.ClockService;
+import org.apache.isis.applib.services.command.Command;
+import org.apache.isis.applib.services.command.CommandDefault;
+import org.apache.isis.applib.services.command.spi.CommandService;
+
+@DomainService(
+        nature = NatureOfService.DOMAIN
+)
+public class CommandServiceDefault implements CommandService {
+
+    @Programmatic
+    @Override
+    public Command create() {
+        return new CommandDefault();
+    }
+
+    @Deprecated
+    @Programmatic
+    @Override
+    public void startTransaction(final Command command, final UUID transactionId) {
+        // nothing to do.
+    }
+
+    @Programmatic
+    @Override
+    public void complete(final Command command) {
+        if(command.getCompletedAt() != null) {
+            // already attempted to complete.
+            // chances are, we're here as the result of a redirect following a previous exception
+            // so just ignore.
+            return;
+        }
+        command.setCompletedAt(clockService.nowAsJavaSqlTimestamp());
+    }
+
+
+    @Programmatic
+    @Override
+    public boolean persistIfPossible(final Command command) {
+        return false;
+    }
+
+    @Inject
+    ClockService clockService;
+
+}

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransaction.java
----------------------------------------------------------------------
diff --git a/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransaction.java b/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransaction.java
index eb68997..3448d46 100644
--- a/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransaction.java
+++ b/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransaction.java
@@ -139,7 +139,7 @@ public class IsisTransaction implements TransactionScopedComponent {
         }
     }
 
-    public static enum State {
+    public enum State {
         /**
          * Started, still in progress.
          * 
@@ -246,18 +246,16 @@ public class IsisTransaction implements TransactionScopedComponent {
      */
     private final Command command;
 
-    /**
-     * Could be null if not configured as a domain service.
-     */
     private final CommandContext commandContext;
+    private final CommandService commandService;
     /**
      * could be null if none has been registered.
      */
-    private final AuditingService3 auditingService3;
+    private final AuditingService3 auditingServiceIfAny;
     /**
      * could be null if none has been registered
      */
-    private final PublishingServiceWithDefaultPayloadFactories publishingService;
+    private final PublishingServiceWithDefaultPayloadFactories publishingServiceIfAny;
 
     /**
      * Will be that of the {@link #command} if not <tt>null</tt>, otherwise will be randomly created.
@@ -274,7 +272,8 @@ public class IsisTransaction implements TransactionScopedComponent {
             final IsisTransactionManager transactionManager,
             final MessageBroker messageBroker,
             final IsisTransactionManager.PersistenceSessionTransactionManagement persistenceSession,
-            final ServicesInjector servicesInjector) {
+            final ServicesInjector servicesInjector,
+            final UUID transactionId) {
         
         ensureThatArg(transactionManager, is(not(nullValue())), "transaction manager is required");
         ensureThatArg(messageBroker, is(not(nullValue())), "message broker is required");
@@ -284,26 +283,19 @@ public class IsisTransaction implements TransactionScopedComponent {
         this.messageBroker = messageBroker;
         this.servicesInjector = servicesInjector;
         
-        this.commandContext = servicesInjector.lookupService(CommandContext.class);
-        this.auditingService3 = servicesInjector.lookupService(AuditingService3.class);
-        this.publishingService = getPublishingServiceIfAny(servicesInjector);
+        this.commandContext = lookupService(CommandContext.class);
+        this.commandService = lookupService(CommandService.class);
+
+        this.auditingServiceIfAny = lookupServiceIfAny(AuditingService3.class);
+        this.publishingServiceIfAny = getPublishingServiceIfAny(servicesInjector);
 
         // determine whether this xactn is taking place in the context of an
         // existing command in which a previous xactn has already occurred.
         // if so, reuse that transactionId.
-        UUID previousTransactionId = null;
-        if(commandContext != null) {
-            command = commandContext.getCommand();
-            previousTransactionId = command.getTransactionId();
-        } else {
-            command = null;
-        }
-        if (previousTransactionId != null) {
-            this.transactionId = previousTransactionId;
-        } else {
-            this.transactionId = UUID.randomUUID();
-        }
-        
+        command = commandContext.getCommand();
+
+        this.transactionId = transactionId;
+
         this.state = State.IN_PROGRESS;
 
         this.persistenceSession = persistenceSession;
@@ -519,7 +511,7 @@ public class IsisTransaction implements TransactionScopedComponent {
 
     protected void doAudit(final Set<Entry<AdapterAndProperty, PreAndPostValues>> changedObjectProperties) {
         try {
-            if(auditingService3 == null) {
+            if(auditingServiceIfAny == null) {
                 return;
             }
 
@@ -538,7 +530,7 @@ public class IsisTransaction implements TransactionScopedComponent {
 
     public void publishActionIfRequired(final String currentUser, final java.sql.Timestamp timestamp) {
 
-        if(publishingService == null) {
+        if(publishingServiceIfAny == null) {
             return;
         }
 
@@ -593,7 +585,7 @@ public class IsisTransaction implements TransactionScopedComponent {
                     parameterNames, parameterTypes, returnType);
 
             final PublishedAction.PayloadFactory payloadFactory = publishedActionFacet.value();
-            publishingService.publishAction(payloadFactory, metadata, currentInvocation, objectStringifier());
+            publishingServiceIfAny.publishAction(payloadFactory, metadata, currentInvocation, objectStringifier());
         } finally {
             // ensures that cannot publish this action more than once
             ActionInvocationFacet.currentInvocation.set(null);
@@ -609,7 +601,7 @@ public class IsisTransaction implements TransactionScopedComponent {
      * @return the adapters that were published (if any were).
      */
     protected List<ObjectAdapter> publishedChangedObjectsIfRequired(final String currentUser, final java.sql.Timestamp timestamp) {
-        if(publishingService == null) {
+        if(publishingServiceIfAny == null) {
             return Collections.emptyList();
         }
         
@@ -631,7 +623,8 @@ public class IsisTransaction implements TransactionScopedComponent {
             final EventMetadata metadata = newEventMetadata(
                     currentUser, timestamp, changeKind, enlistedAdapterClass, enlistedTarget);
 
-            publishingService.publishObject(payloadFactory, metadata, enlistedAdapter, changeKind, objectStringifier());
+            publishingServiceIfAny
+                    .publishObject(payloadFactory, metadata, enlistedAdapter, changeKind, objectStringifier());
         }
         return enlistedAdapters;
     }
@@ -741,7 +734,8 @@ public class IsisTransaction implements TransactionScopedComponent {
 
         final String targetClass = CommandUtil.targetClassNameFor(adapter);
 
-        auditingService3.audit(getTransactionId(), targetClass, target, memberId, propertyId, preValue, postValue, user, timestamp);
+        auditingServiceIfAny
+                .audit(getTransactionId(), targetClass, target, memberId, propertyId, preValue, postValue, user, timestamp);
     }
 
     private static String asString(Object object) {
@@ -812,14 +806,7 @@ public class IsisTransaction implements TransactionScopedComponent {
 
     private void ensureCommandsPersistedIfDirtyXactnAndAnySafeSemanticsHonoured(final Set<Entry<AdapterAndProperty, PreAndPostValues>> changedObjectProperties) {
 
-        final CommandContext commandContext = getServiceOrNull(CommandContext.class);
-        if (commandContext == null) {
-            return;
-        }
         final Command command = commandContext.getCommand();
-        if(command == null) {
-            return;
-        }
 
         // ensure that any changed objects means that the command should be persisted
         final Set<ObjectAdapter> changedAdapters = findChangedAdapters(changedObjectProperties);
@@ -936,42 +923,29 @@ public class IsisTransaction implements TransactionScopedComponent {
     }
 
     private void clearCommandServiceIfConfigured() {
-        completeCommandIfConfigured();
+        completeCommand();
     }
 
 
     private void closeServices() {
         closeOtherApplibServicesIfConfigured();
-        completeCommandIfConfigured();
-    }
-
-    /**
-     * @return - the service, or <tt>null</tt> if no service registered of specified type.
-     */
-    public <T> T getServiceOrNull(Class<T> serviceType) {
-        return servicesInjector.lookupService(serviceType);
+        completeCommand();
     }
 
     private void closeOtherApplibServicesIfConfigured() {
-        ActionInvocationContext bic = getServiceOrNull(ActionInvocationContext.class);
-        if(bic != null) {
+        final ActionInvocationContext actionInvocationContext = lookupService(ActionInvocationContext.class);
+        if(actionInvocationContext != null) {
             Bulk.InteractionContext.current.set(null);
         }
     }
 
-    private void completeCommandIfConfigured() {
-        final CommandContext commandContext = getServiceOrNull(CommandContext.class);
-        if(commandContext != null) {
-            final CommandService commandService = getServiceOrNull(CommandService.class);
-            if(commandService != null) {
-                final Command command = commandContext.getCommand();
-                commandService.complete(command);
+    private void completeCommand() {
+        final Command command = commandContext.getCommand();
 
-                flushActionDomainEvents(command);
-            }
-        }
-    }
+        commandService.complete(command);
 
+        flushActionDomainEvents(command);
+    }
 
     // ////////////////////////////////////////////////////////////////
 
@@ -1014,6 +988,7 @@ public class IsisTransaction implements TransactionScopedComponent {
     // handle exceptions on load, flush or commit
     /////////////////////////////////////////////////////////////////////////
 
+    // SEEMINGLY UNUSED
     @Deprecated
     public void ensureNoAbortCause() {
         Ensure.ensureThatArg(abortCause, is(nullValue()), "abort cause has been set");
@@ -1429,6 +1404,22 @@ public class IsisTransaction implements TransactionScopedComponent {
 
 
     ////////////////////////////////////////////////////////////////////////
+    // Dependencies (lookup)
+    ////////////////////////////////////////////////////////////////////////
+
+    private <T> T lookupService(Class<T> serviceType) {
+        T service = lookupServiceIfAny(serviceType);
+        if(service == null) {
+            throw new IllegalStateException("Could not locate service of type '" + serviceType + "'");
+        }
+        return service;
+    }
+
+    private <T> T lookupServiceIfAny(final Class<T> serviceType) {
+        return servicesInjector.lookupService(serviceType);
+    }
+
+    ////////////////////////////////////////////////////////////////////////
     // Dependencies (from constructor)
     ////////////////////////////////////////////////////////////////////////
 
@@ -1444,5 +1435,4 @@ public class IsisTransaction implements TransactionScopedComponent {
         return persistenceSession;
     }
 
-
 }

http://git-wip-us.apache.org/repos/asf/isis/blob/98a0c2f8/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransactionManager.java
----------------------------------------------------------------------
diff --git a/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransactionManager.java b/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransactionManager.java
index 123f724..486bedb 100644
--- a/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransactionManager.java
+++ b/core/runtime/src/main/java/org/apache/isis/core/runtime/system/transaction/IsisTransactionManager.java
@@ -26,10 +26,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.isis.applib.annotation.Bulk;
-import org.apache.isis.applib.clock.Clock;
+import org.apache.isis.applib.services.clock.ClockService;
 import org.apache.isis.applib.services.command.Command;
 import org.apache.isis.applib.services.command.CommandContext;
-import org.apache.isis.applib.services.command.CommandDefault;
 import org.apache.isis.applib.services.command.spi.CommandService;
 import org.apache.isis.core.commons.authentication.AuthenticationSession;
 import org.apache.isis.core.commons.authentication.MessageBroker;
@@ -45,12 +44,9 @@ import org.apache.isis.core.runtime.services.RequestScopedService;
 import org.apache.isis.core.runtime.system.context.IsisContext;
 import org.apache.isis.core.runtime.system.session.IsisSession;
 
-import static org.apache.isis.core.commons.ensure.Ensure.ensureThatArg;
 import static org.apache.isis.core.commons.ensure.Ensure.ensureThatState;
 import static org.hamcrest.CoreMatchers.is;
-import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.CoreMatchers.notNullValue;
-import static org.hamcrest.CoreMatchers.nullValue;
 
 public class IsisTransactionManager implements SessionScopedComponent {
 
@@ -69,6 +65,11 @@ public class IsisTransactionManager implements SessionScopedComponent {
 
     private final ServicesInjector servicesInjector;
 
+    private final CommandContext commandContext;
+    private final CommandService commandService;
+
+    private final ClockService clockService;
+
 
     // ////////////////////////////////////////////////////////////////
     // constructor
@@ -91,8 +92,14 @@ public class IsisTransactionManager implements SessionScopedComponent {
     public IsisTransactionManager(
             final PersistenceSessionTransactionManagement persistenceSession,
             final ServicesInjector servicesInjector) {
+
         this.persistenceSession = persistenceSession;
         this.servicesInjector = servicesInjector;
+
+        this.commandContext = lookupService(CommandContext.class);
+        this.commandService = lookupService(CommandService.class);
+
+        this.clockService = lookupService(ClockService.class);
     }
 
     public PersistenceSessionTransactionManagement getPersistenceSession() {
@@ -134,8 +141,6 @@ public class IsisTransactionManager implements SessionScopedComponent {
     }
 
 
-    
-
     /**
      * Convenience method returning the {@link org.apache.isis.core.commons.authentication.MessageBroker} of the
      * {@link #getTransaction() current transaction}.
@@ -166,9 +171,14 @@ public class IsisTransactionManager implements SessionScopedComponent {
      * </p>
      */
     public void executeWithinTransaction(final TransactionalClosure closure) {
+        executeWithinTransaction(null, closure);
+    }
+    public void executeWithinTransaction(
+            final Command existingCommandIfAny,
+            final TransactionalClosure closure) {
         final boolean initiallyInTransaction = inTransaction();
         if (!initiallyInTransaction) {
-            startTransaction();
+            startTransaction(existingCommandIfAny);
         }
         try {
             closure.execute();
@@ -194,7 +204,7 @@ public class IsisTransactionManager implements SessionScopedComponent {
     /**
      * Run the supplied {@link Runnable block of code (closure)} in a
      * {@link IsisTransaction transaction}.
-     * 
+     *
      * <p>
      * If a transaction is {@link IsisContext#inTransaction() in progress}, then
      * uses that. Otherwise will {@link #startTransaction() start} a transaction
@@ -208,9 +218,14 @@ public class IsisTransactionManager implements SessionScopedComponent {
      *  </p>
      */
     public <Q> Q executeWithinTransaction(final TransactionalClosureWithReturn<Q> closure) {
+        return executeWithinTransaction(null, closure);
+    }
+    public <Q> Q executeWithinTransaction(
+            final Command existingCommandIfAny,
+            final TransactionalClosureWithReturn<Q> closure) {
         final boolean initiallyInTransaction = inTransaction();
         if (!initiallyInTransaction) {
-            startTransaction();
+            startTransaction(existingCommandIfAny);
         }
         try {
             final Q retVal = closure.execute();
@@ -233,42 +248,19 @@ public class IsisTransactionManager implements SessionScopedComponent {
         return getTransaction() != null && !getTransaction().getState().isComplete();
     }
 
-    // ////////////////////////////////////////////////////////////////
-    // create transaction, + hooks
-    // ////////////////////////////////////////////////////////////////
-
-    /**
-     * Creates a new transaction and saves, to be accessible in
-     * {@link #getTransaction()}.
-     */
-    protected final IsisTransaction createTransaction() {
-        MessageBroker messageBroker = createMessageBroker();
-        return this.transaction = createTransaction(messageBroker, persistenceSession);
-    }
-
-
-    /**
-     * The provided {@link org.apache.isis.core.commons.authentication.MessageBroker} is
-     * obtained from the {@link #createMessageBroker()} hook method.
-     * @param persistenceSession
-     *
-     * @see #createMessageBroker()
-     */
-    private IsisTransaction createTransaction(
-            final MessageBroker messageBroker,
-            final PersistenceSessionTransactionManagement persistenceSession) {
-        ensureThatArg(messageBroker, is(not(nullValue())));
-
-        return new IsisTransaction(this, messageBroker, persistenceSession, servicesInjector);
-    }
-    
 
     // //////////////////////////////////////////////////////
-    // start
+    // startTransaction
     // //////////////////////////////////////////////////////
 
     public synchronized void startTransaction() {
+        startTransaction(null);
+    }
 
+    /**
+     * @param existingCommandIfAny - specifically, a previously persisted background {@link Command}, now being executed by a background execution service.
+     */
+    public void startTransaction(final Command existingCommandIfAny) {
         boolean noneInProgress = false;
         if (getTransaction() == null || getTransaction().getState().isComplete()) {
             noneInProgress = true;
@@ -277,16 +269,35 @@ public class IsisTransactionManager implements SessionScopedComponent {
 
             // note that at this point there may not be an EventBusService initialized.  The PersistenceSession has
             // logic to suppress the posting of the ObjectCreatedEvent for the special case of Command objects.
-            createCommandIfConfigured();
+
+            final Command command;
+            final UUID transactionId;
+
+            if (existingCommandIfAny != null) {
+                command = existingCommandIfAny;
+                transactionId = command.getTransactionId();
+            }
+            else {
+                command = createCommand();
+                transactionId = UUID.randomUUID();
+            }
+
+            initComand(transactionId, command);
+            commandContext.setCommand(command);
+
 
             initOtherApplibServicesIfConfigured();
-            
-            IsisTransaction isisTransaction = createTransaction();
+
+            final MessageBroker messageBroker = MessageBroker.acquire(getAuthenticationSession());
+            this.transaction = new IsisTransaction(
+                    this, messageBroker, persistenceSession, servicesInjector, transactionId);
             transactionLevel = 0;
 
             persistenceSession.startTransaction();
 
-            startTransactionOnCommandIfConfigured(isisTransaction.getTransactionId());
+            // a no-op; the command will have been populated already
+            // in the earlier call to #createCommandAndInitAndSetAsContext(...).
+            commandService.startTransaction(command, transactionId);
         }
 
         transactionLevel++;
@@ -298,7 +309,7 @@ public class IsisTransactionManager implements SessionScopedComponent {
 
     private void initOtherApplibServicesIfConfigured() {
         
-        final Bulk.InteractionContext bic = getServiceOrNull(Bulk.InteractionContext.class);
+        final Bulk.InteractionContext bic = lookupServiceIfAny(Bulk.InteractionContext.class);
         if(bic != null) {
             Bulk.InteractionContext.current.set(bic);
         }
@@ -340,56 +351,27 @@ public class IsisTransactionManager implements SessionScopedComponent {
         }
     }
 
-    private void createCommandIfConfigured() {
-        final CommandContext commandContext = getServiceOrNull(CommandContext.class);
-        if(commandContext == null) {
-            return;
-        } 
-        final CommandService commandService = getServiceOrNull(CommandService.class);
-        final Command command = 
-                commandService != null 
-                    ? commandService.create() 
-                    : new CommandDefault();
-        servicesInjector.injectServicesInto(command);
+    private Command createCommand() {
+        final CommandService commandService = lookupServiceIfAny(CommandService.class);
+        final Command command = commandService.create();
 
-        commandContext.setCommand(command);
-
-        if(command.getTimestamp() == null) {
-            command.setTimestamp(Clock.getTimeAsJavaSqlTimestamp());
-        }
-        if(command.getUser() == null) {
-            command.setUser(getAuthenticationSession().getUserName());
-        }
-        
-        // the remaining properties are set further down the call-stack, if an action is actually performed
-    }
-
-    /**
-     * Called by IsisTransactionManager on start
-     */
-    public void startTransactionOnCommandIfConfigured(final UUID transactionId) {
-        final CommandContext commandContext = getServiceOrNull(CommandContext.class);
-        if(commandContext == null) {
-            return;
-        } 
-        final CommandService commandService = getServiceOrNull(CommandService.class);
-        if(commandService == null) {
-            return;
-        } 
-        final Command command = commandContext.getCommand();
-        commandService.startTransaction(command, transactionId);
+        servicesInjector.injectServicesInto(command);
+        return command;
     }
 
-
     /**
-     * @return - the service, or <tt>null</tt> if no service registered of specified type.
+     * Sets up {@link Command#getTransactionId()}, {@link Command#getUser()} and {@link Command#getTimestamp()}.
+     *
+     * The remaining properties are set further down the call-stack, if an action is actually performed
+     * @param transactionId
      */
-    public <T> T getServiceOrNull(Class<T> serviceType) {
-        return servicesInjector.lookupService(serviceType);
+    private void initComand(final UUID transactionId, final Command command) {
+        command.setTimestamp(clockService.nowAsJavaSqlTimestamp());
+        command.setUser(getAuthenticationSession().getUserName());
+        command.setTransactionId(transactionId);
     }
 
 
-
     // //////////////////////////////////////////////////////
     // flush
     // //////////////////////////////////////////////////////
@@ -554,40 +536,35 @@ public class IsisTransactionManager implements SessionScopedComponent {
     // Hooks
     // //////////////////////////////////////////////////////////////
 
+    // //////////////////////////////////////////////////////
+    // debugging
+    // //////////////////////////////////////////////////////
 
-    /**
-     * Overridable hook, used in
-     * {@link #createTransaction(MessageBroker, PersistenceSessionTransactionManagement)}
-     * 
-     * <p> Called when a new {@link IsisTransaction} is created.
-     */
-    protected MessageBroker createMessageBroker() {
-        return MessageBroker.acquire(getAuthenticationSession());
+    public void debugData(final DebugBuilder debug) {
+        debug.appendln("Transaction", getTransaction());
     }
 
-    // ////////////////////////////////////////////////////////////////
-    // helpers
-    // ////////////////////////////////////////////////////////////////
 
-    // unused?
-    protected void ensureTransactionInProgress() {
-        ensureThatState(getTransaction() != null && !getTransaction().getState().isComplete(), is(true), "No transaction in progress");
-    }
+    ////////////////////////////////////////////////////////////////////////
+    // Dependencies (lookup)
+    ////////////////////////////////////////////////////////////////////////
 
-    // unused?
-    protected void ensureTransactionNotInProgress() {
-        ensureThatState(getTransaction() != null && !getTransaction().getState().isComplete(), is(false), "Transaction in progress");
+    private <T> T lookupService(Class<T> serviceType) {
+        T service = lookupServiceIfAny(serviceType);
+        if(service == null) {
+            throw new IllegalStateException("Could not locate service of type '" + serviceType + "'");
+        }
+        return service;
     }
 
-
-    // //////////////////////////////////////////////////////
-    // debugging
-    // //////////////////////////////////////////////////////
-
-    public void debugData(final DebugBuilder debug) {
-        debug.appendln("Transaction", getTransaction());
+    /**
+     * @return - the service, or <tt>null</tt> if no service registered of specified type.
+     */
+    private <T> T lookupServiceIfAny(Class<T> serviceType) {
+        return servicesInjector.lookupService(serviceType);
     }
 
+
     // ////////////////////////////////////////////////////////////////
     // Dependencies (injected)
     // ////////////////////////////////////////////////////////////////
@@ -615,7 +592,6 @@ public class IsisTransactionManager implements SessionScopedComponent {
     // Dependencies (from context)
     // ////////////////////////////////////////////////////////////////
 
-
     /**
      * Called back by {@link IsisTransaction}.
      */