You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@isis.apache.org by ah...@apache.org on 2022/01/30 11:00:02 UTC

[isis] branch master updated: ISIS-2877: refactoring value recovery from DTO

This is an automated email from the ASF dual-hosted git repository.

ahuber pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/isis.git


The following commit(s) were added to refs/heads/master by this push:
     new 856be40  ISIS-2877: refactoring value recovery from DTO
856be40 is described below

commit 856be40652c36092477b00e42cfe06e79d902e95
Author: Andi Huber <ah...@apache.org>
AuthorDate: Sun Jan 30 11:59:51 2022 +0100

    ISIS-2877: refactoring value recovery from DTO
    
    - simplify SchemaValueMarshaller implementation by factoring out parts
    into an abstract super class
---
 .../isis/applib/util/schema/CommonDtoUtils.java    | 203 ++++++++++++++++-
 .../services/schema/SchemaValueMarshaller.java     |   7 +-
 .../schema/SchemaValueMarshallerAbstract.java      | 244 +++++++++++++++++++++
 .../command/CommandExecutorServiceDefault.java     |   4 +-
 .../command/SchemaValueMarshallerDefault.java      | 204 +++--------------
 .../eventbusservice/EventBusServiceDemoVm.java     |   7 +-
 .../isis/testdomain/value/ValueSemanticsTest.java  |  10 +-
 7 files changed, 493 insertions(+), 186 deletions(-)

diff --git a/api/applib/src/main/java/org/apache/isis/applib/util/schema/CommonDtoUtils.java b/api/applib/src/main/java/org/apache/isis/applib/util/schema/CommonDtoUtils.java
index 4899154..780eb46 100644
--- a/api/applib/src/main/java/org/apache/isis/applib/util/schema/CommonDtoUtils.java
+++ b/api/applib/src/main/java/org/apache/isis/applib/util/schema/CommonDtoUtils.java
@@ -21,18 +21,217 @@ package org.apache.isis.applib.util.schema;
 import java.util.Objects;
 import java.util.Optional;
 
+import org.springframework.lang.Nullable;
+
+import org.apache.isis.applib.jaxb.JavaTimeXMLGregorianCalendarMarshalling;
+import org.apache.isis.applib.value.Blob;
+import org.apache.isis.applib.value.Clob;
+import org.apache.isis.commons.internal.assertions._Assert;
+import org.apache.isis.commons.internal.base._Casts;
+import org.apache.isis.commons.internal.base._Strings;
+import org.apache.isis.commons.internal.context._Context;
+import org.apache.isis.commons.internal.exceptions._Exceptions;
 import org.apache.isis.schema.cmd.v2.MapDto;
+import org.apache.isis.schema.common.v2.BlobDto;
+import org.apache.isis.schema.common.v2.ClobDto;
+import org.apache.isis.schema.common.v2.EnumDto;
+import org.apache.isis.schema.common.v2.ValueDto;
+import org.apache.isis.schema.common.v2.ValueType;
+import org.apache.isis.schema.common.v2.ValueWithTypeDto;
 
+import lombok.NonNull;
+import lombok.SneakyThrows;
 import lombok.val;
+import lombok.experimental.UtilityClass;
 
 /**
  * @since 1.x {@index}
  */
+@UtilityClass
 public final class CommonDtoUtils {
 
+    // -- VALUE RECOVERY
+
+    @SneakyThrows
+    public Object getValueAsObject (
+            final @Nullable ValueWithTypeDto valueDto) {
+
+        if(valueDto==null) {
+            return null;
+        }
+
+        return getValueAsObject(valueDto.getType(), valueDto);
+    }
+
+    @SneakyThrows
+    public Object getValueAsObject (
+            final @NonNull  ValueType valueType,
+            final @Nullable ValueDto valueDto) {
+
+        if(valueDto==null) {
+            return null;
+        }
+
+        switch(valueType) {
+        case STRING:
+            return valueDto.getString();
+        case BYTE:
+            return valueDto.getByte();
+        case SHORT:
+            return valueDto.getShort();
+        case INT:
+            return valueDto.getInt();
+        case LONG:
+            return valueDto.getLong();
+        case FLOAT:
+            return valueDto.getFloat();
+        case DOUBLE:
+            return valueDto.getDouble();
+        case BOOLEAN:
+            return valueDto.isBoolean();
+        case CHAR:
+            final String aChar = valueDto.getChar();
+            if(_Strings.isNullOrEmpty(aChar)) { return null; }
+            return aChar.charAt(0);
+        case BIG_DECIMAL:
+            return valueDto.getBigDecimal();
+        case BIG_INTEGER:
+            return valueDto.getBigInteger();
+        case LOCAL_DATE:
+            return JavaTimeXMLGregorianCalendarMarshalling.toLocalDate(valueDto.getLocalDate());
+        case LOCAL_TIME:
+            return JavaTimeXMLGregorianCalendarMarshalling.toLocalTime(valueDto.getLocalTime());
+        case LOCAL_DATE_TIME:
+            return JavaTimeXMLGregorianCalendarMarshalling.toLocalDateTime(valueDto.getLocalDateTime());
+        case OFFSET_DATE_TIME:
+            return JavaTimeXMLGregorianCalendarMarshalling.toOffsetDateTime(valueDto.getOffsetDateTime());
+        case OFFSET_TIME:
+            return JavaTimeXMLGregorianCalendarMarshalling.toOffsetTime(valueDto.getOffsetTime());
+        case ZONED_DATE_TIME:
+            return JavaTimeXMLGregorianCalendarMarshalling.toZonedDateTime(valueDto.getZonedDateTime());
+        case ENUM:
+            final EnumDto enumDto = valueDto.getEnum();
+            final String enumType = enumDto.getEnumType();
+            @SuppressWarnings("rawtypes")
+            final Class<? extends Enum> enumClass =
+                    _Casts.uncheckedCast(_Context.loadClassAndInitialize(enumType));
+            return Enum.valueOf(_Casts.uncheckedCast(enumClass), enumDto.getEnumName());
+        case REFERENCE:
+            return valueDto.getReference();
+        case BLOB:
+            final BlobDto blobDto = valueDto.getBlob();
+            return new Blob(blobDto.getName(), blobDto.getMimeType(), blobDto.getBytes());
+        case CLOB:
+            final ClobDto clobDto = valueDto.getClob();
+            return new Clob(clobDto.getName(), clobDto.getMimeType(), clobDto.getChars());
+        case VOID:
+            return null;
+        default:
+            throw _Exceptions.unmatchedCase(valueType);
+        }
+
+    }
+
+    // -- LIFTING
+
+    public ValueWithTypeDto toValueWithTypeDto(
+            final @NonNull  ValueType valueType,
+            final @Nullable ValueDto valueDto) {
+
+        if(valueDto instanceof ValueWithTypeDto) {
+            val downCast = (ValueWithTypeDto) valueDto;
+            _Assert.assertEquals(valueType, downCast.getType());
+            return downCast;
+        }
+
+        val dto = new ValueWithTypeDto();
+        dto.setType(valueType);
+
+        if(valueDto==null) {
+            return dto; // null to empty
+        }
+
+        switch(valueType) {
+        case BIG_DECIMAL:
+            dto.setBigDecimal(valueDto.getBigDecimal());
+            break;
+        case BIG_INTEGER:
+            dto.setBigInteger(valueDto.getBigInteger());
+            break;
+        case BLOB:
+            dto.setBlob(valueDto.getBlob());
+            break;
+        case BOOLEAN:
+            dto.setBoolean(valueDto.isBoolean());
+            break;
+        case BYTE:
+            dto.setByte(valueDto.getByte());
+            break;
+        case CHAR:
+            dto.setChar(valueDto.getChar());
+            break;
+        case CLOB:
+            dto.setClob(valueDto.getClob());
+            break;
+        case COLLECTION:
+            dto.setCollection(valueDto.getCollection());
+            break;
+        case COMPOSITE:
+            dto.setComposite(valueDto.getComposite());
+            break;
+        case DOUBLE:
+            dto.setDouble(valueDto.getDouble());
+            break;
+        case ENUM:
+            dto.setEnum(valueDto.getEnum());
+            break;
+        case FLOAT:
+            dto.setFloat(valueDto.getFloat());
+            break;
+        case INT:
+            dto.setInt(valueDto.getInt());
+            break;
+        case LOCAL_DATE:
+            dto.setLocalDate(valueDto.getLocalDate());
+            break;
+        case LOCAL_DATE_TIME:
+            dto.setLocalDateTime(valueDto.getLocalDateTime());
+            break;
+        case LOCAL_TIME:
+            dto.setLocalTime(valueDto.getLocalTime());
+            break;
+        case LONG:
+            dto.setLong(valueDto.getLong());
+            break;
+        case OFFSET_DATE_TIME:
+            dto.setOffsetDateTime(valueDto.getOffsetDateTime());
+            break;
+        case OFFSET_TIME:
+            dto.setOffsetTime(valueDto.getOffsetTime());
+            break;
+        case REFERENCE:
+            dto.setReference(valueDto.getReference());
+            break;
+        case SHORT:
+            dto.setShort(valueDto.getShort());
+            break;
+        case STRING:
+            dto.setString(valueDto.getString());
+            break;
+        case VOID:
+            break;
+        case ZONED_DATE_TIME:
+            dto.setZonedDateTime(valueDto.getZonedDateTime());
+            break;
+        default:
+            throw _Exceptions.unmatchedCase(valueType);
+        }
+        return dto;
+    }
+
     // -- MAP-DTO SUPPORT
 
-    static String getMapValue(final MapDto mapDto, final String key) {
+    public String getMapValue(final MapDto mapDto, final String key) {
         if(mapDto == null) {
             return null;
         }
@@ -40,7 +239,7 @@ public final class CommonDtoUtils {
         return entryIfAny.map(MapDto.Entry::getValue).orElse(null);
     }
 
-    static void putMapKeyValue(final MapDto mapDto, final String key, final String value) {
+    public void putMapKeyValue(final MapDto mapDto, final String key, final String value) {
         if(mapDto == null) {
             return;
         }
diff --git a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshaller.java b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshaller.java
index 1dd9f88..4df9f05 100644
--- a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshaller.java
+++ b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshaller.java
@@ -64,16 +64,17 @@ public interface SchemaValueMarshaller {
     /**
      * Recovers a property value, using {@link ValueSemanticsProvider}
      * for corresponding <i>Property</i>.
+     * Cardinality {@code 0..1}
      */
-    ManagedObject recoverValueFrom(@NonNull PropertyDto propertyDto);
+    ManagedObject recoverPropertyFrom(@NonNull PropertyDto propertyDto);
 
     /**
      * Recovers a parameter value, using {@link ValueSemanticsProvider}
-     * for corresponding <i>Action Parameter</i>.
+     * for corresponding <i>Action Parameter</i>. Cardinality {@code 0..n}
      * <p>
      * Packed up if non-scalar.
      */
-    ManagedObject recoverValuesFrom(@NonNull Identifier paramIdentifier, @NonNull ParamDto paramDto);
+    ManagedObject recoverParameterFrom(@NonNull Identifier paramIdentifier, @NonNull ParamDto paramDto);
 
     // -- RECORD VALUES INTO DTO
 
diff --git a/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshallerAbstract.java b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshallerAbstract.java
new file mode 100644
index 0000000..462dcca
--- /dev/null
+++ b/core/metamodel/src/main/java/org/apache/isis/core/metamodel/services/schema/SchemaValueMarshallerAbstract.java
@@ -0,0 +1,244 @@
+/*
+ *  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.metamodel.services.schema;
+
+import java.util.ArrayList;
+import java.util.Optional;
+
+import org.springframework.lang.Nullable;
+
+import org.apache.isis.applib.Identifier;
+import org.apache.isis.applib.Identifier.Type;
+import org.apache.isis.applib.services.bookmark.Bookmark;
+import org.apache.isis.applib.util.schema.CommonDtoUtils;
+import org.apache.isis.applib.value.semantics.ValueSemanticsProvider;
+import org.apache.isis.applib.value.semantics.ValueSemanticsResolver;
+import org.apache.isis.commons.collections.Can;
+import org.apache.isis.commons.collections.Cardinality;
+import org.apache.isis.commons.internal.assertions._Assert;
+import org.apache.isis.commons.internal.base._NullSafe;
+import org.apache.isis.core.metamodel.facets.actions.action.invocation.IdentifierUtil;
+import org.apache.isis.core.metamodel.objectmanager.load.ObjectLoader;
+import org.apache.isis.core.metamodel.spec.ManagedObject;
+import org.apache.isis.core.metamodel.spec.ObjectSpecification;
+import org.apache.isis.core.metamodel.spec.PackedManagedObject;
+import org.apache.isis.core.metamodel.spec.feature.ObjectFeature;
+import org.apache.isis.core.metamodel.specloader.SpecificationLoader;
+import org.apache.isis.schema.cmd.v2.ActionDto;
+import org.apache.isis.schema.cmd.v2.ParamDto;
+import org.apache.isis.schema.cmd.v2.PropertyDto;
+import org.apache.isis.schema.common.v2.CollectionDto;
+import org.apache.isis.schema.common.v2.OidDto;
+import org.apache.isis.schema.common.v2.ValueDto;
+import org.apache.isis.schema.common.v2.ValueType;
+import org.apache.isis.schema.common.v2.ValueWithTypeDto;
+import org.apache.isis.schema.ixn.v2.ActionInvocationDto;
+
+import lombok.NonNull;
+import lombok.Value;
+import lombok.val;
+
+public abstract class SchemaValueMarshallerAbstract
+implements SchemaValueMarshaller {
+
+    @Value(staticConstructor = "of")
+    public static class ValueTypeHelper {
+
+        private final @NonNull ObjectFeature feature;
+        private final @Nullable ValueSemanticsProvider<?> semantics;
+
+        public ObjectSpecification getElementType() {
+            return feature.getElementType();
+        }
+        public ValueType getSchemaValueType() {
+            return semantics.getSchemaValueType();
+        }
+    }
+
+    // -- RECOVER IDENTIFIERS
+
+    @Override
+    public final Identifier actionIdentifier(final @NonNull ActionDto actionDto) {
+        return IdentifierUtil.memberIdentifierFor(getSpecificationLoader(),
+                Type.ACTION,
+                actionDto.getLogicalMemberIdentifier());
+    }
+
+    @Override
+    public final Identifier actionIdentifier(final @NonNull ActionInvocationDto actionInvocationDto) {
+        return IdentifierUtil.memberIdentifierFor(getSpecificationLoader(),
+                Type.ACTION,
+                actionInvocationDto.getLogicalMemberIdentifier());
+    }
+
+    @Override
+    public final Identifier propertyIdentifier(final @NonNull PropertyDto propertyDto) {
+        return IdentifierUtil.memberIdentifierFor(getSpecificationLoader(),
+                Type.PROPERTY_OR_COLLECTION,
+                propertyDto.getLogicalMemberIdentifier());
+    }
+
+    // -- RECOVER VALUES FROM DTO
+
+    @Override
+    public final ManagedObject recoverReferenceFrom(
+            final @NonNull OidDto oidDto) {
+        val bookmark = Bookmark.forOidDto(oidDto);
+        val spec = getSpecificationLoader().specForLogicalTypeNameElseFail(bookmark.getLogicalTypeName());
+        val loadRequest = ObjectLoader.Request.of(spec, bookmark);
+        return spec.getMetaModelContext().getObjectManager().loadObject(loadRequest);
+    }
+
+    @Override
+    public final ManagedObject recoverPropertyFrom(
+            final @NonNull PropertyDto propertyDto) {
+        final Identifier propertyIdentifier = propertyIdentifier(propertyDto);
+        val valueWithTypeDto = propertyDto.getNewValue();
+        return recoverValueOrReference(propertyIdentifier, valueWithTypeDto, Cardinality.ONE);
+    }
+
+    @Override
+    public final ManagedObject recoverParameterFrom(
+            final @NonNull Identifier paramIdentifier,
+            final @NonNull ParamDto paramDto) {
+
+        val cardinalityConstraint = paramDto.getType().equals(ValueType.COLLECTION)
+                ? Cardinality.MULTIPLE
+                : Cardinality.ONE;
+        return recoverValueOrReference(paramIdentifier, paramDto, cardinalityConstraint);
+    }
+
+    // -- HELPER
+
+    private ManagedObject recoverValueOrReference(
+            final Identifier featureIdentifier,
+            final ValueWithTypeDto valueWithTypeDto,
+            final Cardinality cardinalityConstraint) {
+
+        val feature = getSpecificationLoader().loadFeatureElseFail(featureIdentifier);
+
+        if(valueWithTypeDto==null
+                || (valueWithTypeDto.isSetNull()
+                    && valueWithTypeDto.isNull())) {
+            return cardinalityConstraint.isMultiple()
+                    ? PackedManagedObject.pack(feature.getElementType(), Can.empty())
+                    : ManagedObject.empty(feature.getElementType());
+        }
+
+        @SuppressWarnings({ "unchecked", "rawtypes" })
+        final Optional<ValueSemanticsProvider<?>> preferredValueSemantics = getValueSemanticsResolver()
+                .selectValueSemantics(
+                        featureIdentifier,
+                        (Class)feature.getElementType().getCorrespondingClass())
+                .getFirst();
+
+        val recoveredValueOrReference = preferredValueSemantics
+            .map(valueSemantics->ValueTypeHelper.of(feature, valueSemantics))
+            .map(valueTypeHelper->recoverValue(valueTypeHelper, valueWithTypeDto, cardinalityConstraint))
+            // assume reference otherwise
+            .orElseGet(()->recoverReference(feature, valueWithTypeDto, cardinalityConstraint));
+
+        return recoveredValueOrReference;
+    }
+
+    // -- LOW LEVEL IMPLEMENTATION
+
+    /**
+     * References, collections and {@code null} are already dealt with.
+     * Implementations only need to consider a non-empty scalar value-type.
+     */
+    protected abstract ManagedObject recoverScalarValue(
+            final @NonNull ValueTypeHelper valueTypeHelper,
+            final @NonNull ValueWithTypeDto valueDto);
+
+    protected ManagedObject recoverValue(
+            final @NonNull ValueTypeHelper valueTypeHelper,
+            final @NonNull ValueWithTypeDto valueDto,
+            final @NonNull Cardinality cardinalityConstraint) {
+
+        return cardinalityConstraint.isMultiple()
+                ? PackedManagedObject.pack(
+                        valueTypeHelper.getElementType(),
+                        recoverCollectionOfValues(valueTypeHelper, valueDto.getCollection()))
+                : recoverScalarValue(valueTypeHelper, valueDto);
+//                    valueDto.getReference()!=null
+//                        ? recoverScalarValue(valueTypeHelper, valueDto)
+//                        : ManagedObject.empty(valueTypeHelper.getElementType());
+    }
+
+    protected ManagedObject recoverReference(
+            final @NonNull ObjectFeature feature,
+            final @NonNull ValueDto valueDto,
+            final @NonNull Cardinality cardinalityConstraint) {
+
+        return cardinalityConstraint.isMultiple()
+                ? PackedManagedObject.pack(
+                        feature.getElementType(),
+                        recoverCollectionOfReferences(valueDto.getCollection()))
+                : recoverReferenceFrom(valueDto.getReference());
+    }
+
+    protected Can<ManagedObject> recoverCollectionOfValues(
+            final ValueTypeHelper valueTypeHelper,
+            final CollectionDto collectionDto) {
+
+        _Assert.assertEquals(valueTypeHelper.getSchemaValueType(), collectionDto.getType());
+
+        if(_NullSafe.isEmpty(collectionDto.getValue())) {
+            return Can.empty();
+        }
+
+        val elementDtos = collectionDto.getValue();
+        val list = new ArrayList<ManagedObject>(elementDtos.size());
+
+        for(val _elementDto : elementDtos) {
+
+            val elementDto = CommonDtoUtils
+                    .toValueWithTypeDto(valueTypeHelper.getSchemaValueType(), _elementDto);
+            val cardinalityConstraint = elementDto.getCollection()!=null
+                    ? Cardinality.MULTIPLE
+                    : Cardinality.ONE;
+            list.add(recoverValue(valueTypeHelper, elementDto, cardinalityConstraint));
+        }
+        return Can.ofCollection(list);
+    }
+
+    protected Can<ManagedObject> recoverCollectionOfReferences(
+            final CollectionDto collectionDto) {
+
+        if(_NullSafe.isEmpty(collectionDto.getValue())) {
+            return Can.empty();
+        }
+
+        val elementDtos = collectionDto.getValue();
+        val list = new ArrayList<ManagedObject>(elementDtos.size());
+
+        for(val elementDto : elementDtos) {
+            list.add(recoverReferenceFrom(elementDto.getReference()));
+        }
+        return Can.ofCollection(list);
+    }
+
+
+    // -- DEPENDENCIES
+
+    protected abstract SpecificationLoader getSpecificationLoader();
+    protected abstract ValueSemanticsResolver getValueSemanticsResolver();
+
+}
diff --git a/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/CommandExecutorServiceDefault.java b/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/CommandExecutorServiceDefault.java
index 0a7f8c9..4e279e0 100644
--- a/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/CommandExecutorServiceDefault.java
+++ b/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/CommandExecutorServiceDefault.java
@@ -240,7 +240,7 @@ public class CommandExecutorServiceDefault implements CommandExecutorService {
 
                 final OneToOneAssociation property = findOneToOneAssociation(targetAdapter, memberId);
 
-                val newValueAdapter = valueMarshaller.recoverValueFrom(propertyDto);
+                val newValueAdapter = valueMarshaller.recoverPropertyFrom(propertyDto);
 
                 property.set(targetAdapter, newValueAdapter, InteractionInitiatedBy.FRAMEWORK);
 
@@ -353,7 +353,7 @@ public class CommandExecutorServiceDefault implements CommandExecutorService {
 
         return streamParamDtosFrom(actionDto)
                 .map(IndexedFunction.zeroBased((i, paramDto)->
-                    valueMarshaller.recoverValuesFrom(actionIdentifier.withParameterIndex(i), paramDto)))
+                    valueMarshaller.recoverParameterFrom(actionIdentifier.withParameterIndex(i), paramDto)))
                 .collect(Can.toCan());
     }
 
diff --git a/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/SchemaValueMarshallerDefault.java b/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/SchemaValueMarshallerDefault.java
index edad5fc..b4a446a 100644
--- a/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/SchemaValueMarshallerDefault.java
+++ b/core/runtimeservices/src/main/java/org/apache/isis/core/runtimeservices/command/SchemaValueMarshallerDefault.java
@@ -8,7 +8,6 @@ import java.time.LocalTime;
 import java.time.OffsetDateTime;
 import java.time.OffsetTime;
 import java.time.ZonedDateTime;
-import java.util.ArrayList;
 
 import javax.annotation.Priority;
 import javax.inject.Inject;
@@ -19,10 +18,9 @@ import org.springframework.lang.Nullable;
 import org.springframework.stereotype.Service;
 
 import org.apache.isis.applib.Identifier;
-import org.apache.isis.applib.Identifier.Type;
 import org.apache.isis.applib.annotation.PriorityPrecedence;
 import org.apache.isis.applib.jaxb.JavaTimeXMLGregorianCalendarMarshalling;
-import org.apache.isis.applib.services.bookmark.Bookmark;
+import org.apache.isis.applib.util.schema.CommonDtoUtils;
 import org.apache.isis.applib.value.Blob;
 import org.apache.isis.applib.value.Clob;
 import org.apache.isis.applib.value.semantics.Converter;
@@ -32,28 +30,20 @@ import org.apache.isis.applib.value.semantics.ValueSemanticsResolver;
 import org.apache.isis.commons.collections.Can;
 import org.apache.isis.commons.internal.assertions._Assert;
 import org.apache.isis.commons.internal.base._Casts;
-import org.apache.isis.commons.internal.base._NullSafe;
-import org.apache.isis.commons.internal.base._Strings;
-import org.apache.isis.commons.internal.context._Context;
 import org.apache.isis.commons.internal.exceptions._Exceptions;
 import org.apache.isis.core.metamodel.facetapi.FeatureType;
-import org.apache.isis.core.metamodel.facets.actions.action.invocation.IdentifierUtil;
-import org.apache.isis.core.metamodel.objectmanager.load.ObjectLoader;
-import org.apache.isis.core.metamodel.services.schema.SchemaValueMarshaller;
+import org.apache.isis.core.metamodel.services.schema.SchemaValueMarshallerAbstract;
 import org.apache.isis.core.metamodel.spec.ManagedObject;
-import org.apache.isis.core.metamodel.spec.ManagedObjects;
 import org.apache.isis.core.metamodel.spec.ObjectSpecification;
 import org.apache.isis.core.metamodel.spec.feature.ObjectActionParameter;
 import org.apache.isis.core.metamodel.spec.feature.OneToOneAssociation;
 import org.apache.isis.core.metamodel.specloader.SpecificationLoader;
-import org.apache.isis.schema.cmd.v2.ActionDto;
 import org.apache.isis.schema.cmd.v2.ParamDto;
 import org.apache.isis.schema.cmd.v2.PropertyDto;
 import org.apache.isis.schema.common.v2.BlobDto;
 import org.apache.isis.schema.common.v2.ClobDto;
 import org.apache.isis.schema.common.v2.CollectionDto;
 import org.apache.isis.schema.common.v2.EnumDto;
-import org.apache.isis.schema.common.v2.OidDto;
 import org.apache.isis.schema.common.v2.TypedTupleDto;
 import org.apache.isis.schema.common.v2.ValueDto;
 import org.apache.isis.schema.common.v2.ValueType;
@@ -62,7 +52,6 @@ import org.apache.isis.schema.ixn.v2.ActionInvocationDto;
 
 import lombok.NonNull;
 import lombok.RequiredArgsConstructor;
-import lombok.SneakyThrows;
 import lombok.Value;
 import lombok.val;
 
@@ -72,56 +61,37 @@ import lombok.val;
 @Qualifier("Default")
 @RequiredArgsConstructor
 public class SchemaValueMarshallerDefault
-implements SchemaValueMarshaller {
+extends SchemaValueMarshallerAbstract {
 
     @Inject private ValueSemanticsResolver valueSemanticsResolver;
     @Inject private SpecificationLoader specLoader;
 
-    // -- RECOVER IDENTIFIERS
+    // -- RECOVER VALUES FROM DTO
 
     @Override
-    public Identifier actionIdentifier(final @NonNull ActionDto actionDto) {
-        return IdentifierUtil.memberIdentifierFor(specLoader,
-                Type.ACTION,
-                actionDto.getLogicalMemberIdentifier());
-    }
+    protected ManagedObject recoverScalarValue(
+            @NonNull final ValueTypeHelper valueTypeHelper,
+            @NonNull final ValueWithTypeDto valueDto) {
 
-    @Override
-    public Identifier actionIdentifier(final @NonNull ActionInvocationDto actionInvocationDto) {
-        return IdentifierUtil.memberIdentifierFor(specLoader,
-                Type.ACTION,
-                actionInvocationDto.getLogicalMemberIdentifier());
-    }
+        val valueAsObject = CommonDtoUtils.getValueAsObject(valueDto);
 
-    @Override
-    public Identifier propertyIdentifier(final @NonNull PropertyDto propertyDto) {
-        return IdentifierUtil.memberIdentifierFor(specLoader,
-                Type.PROPERTY_OR_COLLECTION,
-                propertyDto.getLogicalMemberIdentifier());
-    }
+        if(valueAsObject==null) {
+            return ManagedObject.empty(valueTypeHelper.getElementType());
+        }
 
-    // -- RECOVER VALUES FROM DTO
+        val feature = valueTypeHelper.getFeature();
+        val elementSpec = valueTypeHelper.getElementType();
 
-    @Override
-    public ManagedObject recoverReferenceFrom(final @NonNull OidDto oidDto) {
-        val bookmark = Bookmark.forOidDto(oidDto);
-        val spec = specLoader.specForLogicalTypeName(bookmark.getLogicalTypeName()).orElse(null);
-        val loadRequest = ObjectLoader.Request.of(spec, bookmark);
-        return spec.getMetaModelContext().getObjectManager().loadObject(loadRequest);
-    }
+        final ValueTypeWrapper<?> valueWrapper = wrap(feature.getFeatureIdentifier(), elementSpec);
 
-    @Override
-    public ManagedObject recoverValueFrom(final @NonNull PropertyDto propertyDto) {
-        val identifier = propertyIdentifier(propertyDto);
-        val valueWithTypeDto = propertyDto.getNewValue();
-        return recoverValue(identifier, valueWithTypeDto);
-    }
+        if(valueDto.getComposite()!=null) {
+            return ManagedObject.of(elementSpec,
+                    fromTypedTuple(valueDto.getComposite()));
+        }
 
-    @Override
-    public ManagedObject recoverValuesFrom(
-            final @NonNull Identifier paramIdentifier,
-            final @NonNull ParamDto paramDto) {
-        return recoverValue(paramIdentifier, paramDto);
+        val recoveredValue = ManagedObject.of(elementSpec,
+                valueWrapper.fromFundamentalValue(CommonDtoUtils.getValueAsObject(valueDto)));
+        return recoveredValue;
     }
 
     // -- RECORD VALUES INTO DTO
@@ -256,11 +226,6 @@ implements SchemaValueMarshaller {
                     : null;
         }
 
-        public T fromTypedTuple(final TypedTupleDto typedTupleDto) {
-            // FIXME[ISIS-2877] implement
-            return null;
-        }
-
         public TypedTupleDto toTypedTupleDto(final Object pojo) {
             // FIXME[ISIS-2877] implement
             return null;
@@ -476,128 +441,21 @@ implements SchemaValueMarshaller {
 
     // -- HELPER - RECOVERY
 
-    private ManagedObject recoverValue(
-            final Identifier featureIdentifier,
-            final ValueWithTypeDto valueWithTypeDto) {
-
-        val feature = specLoader.loadFeatureElseFail(featureIdentifier);
-        val desiredTypeSpec = feature.getElementType();
-
-        if(valueWithTypeDto==null
-                || (valueWithTypeDto.isSetNull()
-                    && valueWithTypeDto.isNull())) {
-            return ManagedObject.empty(desiredTypeSpec);
-        }
-
-        final ValueTypeWrapper<?> valueWrapper = wrap(featureIdentifier, desiredTypeSpec);
-        return recoverValue(valueWithTypeDto, valueWrapper);
+    private Object fromTypedTuple(final TypedTupleDto typedTupleDto) {
+        // FIXME[ISIS-2877] implement
+        return null;
     }
 
-    private ManagedObject recoverValue(
-            final ValueDto valueDto,
-            final @NonNull ValueTypeWrapper<?> valueWrapper) {
-
-        val elementSpec = valueWrapper.getSpec();
-
-        if(valueDto.getCollection()!=null) {
-            return ManagedObjects.pack(elementSpec, recoverCollection(valueWrapper, valueDto.getCollection()));
-        }
-
-        if(valueDto.getComposite()!=null) {
-            return ManagedObject.of(elementSpec,
-                    valueWrapper.fromTypedTuple(valueDto.getComposite()));
-        }
-
-        return ManagedObject.of(elementSpec,
-                valueWrapper.fromFundamentalValue(recoverFundamentalValue(valueDto, valueWrapper)));
-    }
-
-    @SneakyThrows
-    private Object recoverFundamentalValue(
-            final ValueDto valueDto,
-            final ValueTypeWrapper<?> valueWrapper) {
-
-        val elementType = valueWrapper.getValueType();
-
-        switch(elementType) {
-        case STRING:
-            return valueDto.getString();
-        case BYTE:
-            return valueDto.getByte();
-        case SHORT:
-            return valueDto.getShort();
-        case INT:
-            return valueDto.getInt();
-        case LONG:
-            return valueDto.getLong();
-        case FLOAT:
-            return valueDto.getFloat();
-        case DOUBLE:
-            return valueDto.getDouble();
-        case BOOLEAN:
-            return valueDto.isBoolean();
-        case CHAR:
-            final String aChar = valueDto.getChar();
-            if(_Strings.isNullOrEmpty(aChar)) { return null; }
-            return aChar.charAt(0);
-        case BIG_DECIMAL:
-            return valueDto.getBigDecimal();
-        case BIG_INTEGER:
-            return valueDto.getBigInteger();
-        case LOCAL_DATE:
-            return JavaTimeXMLGregorianCalendarMarshalling.toLocalDate(valueDto.getLocalDate());
-        case LOCAL_TIME:
-            return JavaTimeXMLGregorianCalendarMarshalling.toLocalTime(valueDto.getLocalTime());
-        case LOCAL_DATE_TIME:
-            return JavaTimeXMLGregorianCalendarMarshalling.toLocalDateTime(valueDto.getLocalDateTime());
-        case OFFSET_DATE_TIME:
-            return JavaTimeXMLGregorianCalendarMarshalling.toOffsetDateTime(valueDto.getOffsetDateTime());
-        case OFFSET_TIME:
-            return JavaTimeXMLGregorianCalendarMarshalling.toOffsetTime(valueDto.getOffsetTime());
-        case ZONED_DATE_TIME:
-            return JavaTimeXMLGregorianCalendarMarshalling.toZonedDateTime(valueDto.getZonedDateTime());
-        case ENUM:
-            final EnumDto enumDto = valueDto.getEnum();
-            final String enumType = enumDto.getEnumType();
-            @SuppressWarnings("rawtypes")
-            final Class<? extends Enum> enumClass =
-                    _Casts.uncheckedCast(_Context.loadClassAndInitialize(enumType));
-            return Enum.valueOf(_Casts.uncheckedCast(enumClass), enumDto.getEnumName());
-        case REFERENCE:
-            return valueDto.getReference();
-        case BLOB:
-            final BlobDto blobDto = valueDto.getBlob();
-            return new Blob(blobDto.getName(), blobDto.getMimeType(), blobDto.getBytes());
-        case CLOB:
-            final ClobDto clobDto = valueDto.getClob();
-            return new Clob(clobDto.getName(), clobDto.getMimeType(), clobDto.getChars());
-        case VOID:
-            return null;
-        default:
-            throw _Exceptions.unmatchedCase(elementType);
-        }
+    // -- DEPENDENCIES
 
+    @Override
+    protected final SpecificationLoader getSpecificationLoader() {
+        return specLoader;
     }
 
-    private Can<ManagedObject> recoverCollection(
-            final ValueTypeWrapper<?> valueWrapper,
-            final CollectionDto collectionDto) {
-
-        _Assert.assertEquals(valueWrapper.getValueType(), collectionDto.getType());
-
-        if(_NullSafe.isEmpty(collectionDto.getValue())) {
-            return Can.empty();
-        }
-        val list = new ArrayList<ManagedObject>();
-
-        for(val elementDto : collectionDto.getValue()) {
-            if(elementDto instanceof ValueWithTypeDto) {
-                _Assert.assertEquals(valueWrapper, ((ValueWithTypeDto)elementDto).getType(),
-                        "mixing types not supported");
-            }
-            list.add(recoverValue(elementDto, valueWrapper));
-        }
-        return Can.ofCollection(list);
+    @Override
+    protected ValueSemanticsResolver getValueSemanticsResolver() {
+        return valueSemanticsResolver;
     }
 
 }
diff --git a/examples/demo/domain/src/main/java/demoapp/dom/services/core/eventbusservice/EventBusServiceDemoVm.java b/examples/demo/domain/src/main/java/demoapp/dom/services/core/eventbusservice/EventBusServiceDemoVm.java
index 3080994..39a8835 100644
--- a/examples/demo/domain/src/main/java/demoapp/dom/services/core/eventbusservice/EventBusServiceDemoVm.java
+++ b/examples/demo/domain/src/main/java/demoapp/dom/services/core/eventbusservice/EventBusServiceDemoVm.java
@@ -22,6 +22,7 @@ import java.util.List;
 
 import javax.inject.Inject;
 
+import org.apache.isis.applib.ViewModel;
 import org.apache.isis.applib.annotation.Action;
 import org.apache.isis.applib.annotation.ActionLayout;
 import org.apache.isis.applib.annotation.ActionLayout.Position;
@@ -48,7 +49,11 @@ public class EventBusServiceDemoVm implements HasAsciiDocDescription {
         return eventLogEntryRepository.listAll();
     }
 
-    public static class UiButtonEvent {}
+    @DomainObject(nature = Nature.VIEW_MODEL, logicalTypeName = "demo.EventBusServiceDemoVm.UiButtonEvent")
+    public static class UiButtonEvent implements ViewModel {
+        @Override public String viewModelMemento() { return ""; }
+        @Override public void viewModelInit(final String memento) { }
+    }
 
     @ActionLayout(
             describedAs = "Writes a new EventLog entry to the persistent eventlog.",
diff --git a/regressiontests/stable-value/src/test/java/org/apache/isis/testdomain/value/ValueSemanticsTest.java b/regressiontests/stable-value/src/test/java/org/apache/isis/testdomain/value/ValueSemanticsTest.java
index 439d8f1..05c905c 100644
--- a/regressiontests/stable-value/src/test/java/org/apache/isis/testdomain/value/ValueSemanticsTest.java
+++ b/regressiontests/stable-value/src/test/java/org/apache/isis/testdomain/value/ValueSemanticsTest.java
@@ -35,6 +35,10 @@ import org.junit.jupiter.params.provider.MethodSource;
 import org.springframework.boot.test.context.SpringBootTest;
 import org.springframework.test.context.TestPropertySource;
 
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
 import org.apache.isis.applib.annotation.Where;
 import org.apache.isis.applib.locale.UserLocale;
 import org.apache.isis.applib.services.command.Command;
@@ -62,10 +66,6 @@ import org.apache.isis.testdomain.model.valuetypes.ValueTypeExampleService;
 import org.apache.isis.testdomain.model.valuetypes.ValueTypeExampleService.Scenario;
 import org.apache.isis.testdomain.value.ValueSemanticsTester.PropertyInteractionProbe;
 
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
 import lombok.val;
 
 @SpringBootTest(
@@ -197,7 +197,7 @@ class ValueSemanticsTest {
                         val propertyDto = (PropertyDto)command.getCommandDto().getMember();
                         val newValueRecordedDto = propertyDto.getNewValue();
 
-                        val newValueRecorded = valueMarshaller.recoverValueFrom(propertyDto);
+                        val newValueRecorded = valueMarshaller.recoverPropertyFrom(propertyDto);
                         assertNotNull(newValueRecorded);
 
                         assertEquals(valueType, newValueRecorded.getSpecification().getCorrespondingClass(), ()->