You are viewing a plain text version of this content. The canonical link for it is here.
Posted to server-dev@james.apache.org by rc...@apache.org on 2020/08/21 02:26:48 UTC

[james-project] branch master updated (32a6ea9 -> ccfbe53)

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

rcordier pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git.


    from 32a6ea9  JAMES-3357 Handle rights upon Mailbox/set create
     new bb4029d  JAMES-3361 JMAP Draft: sharee should not be able to modify mailbox rights
     new f5d397c  JAMES-3361 ImapHostSystem should use owner's session to set rights for mailbox
     new 0b2d842  JAMES-3359 Mailbox/set update name support
     new 2b1d2e7  JAMES-3359 Mailbox/set update should accept creationIds
     new 9fb1d3c  JAMES-3359 Mailbox/set update should handle Mailbox level errors
     new fa160b8  JAMES-3359 Mailbox/set update should handle subscription
     new b442797  JAMES-3359 Mailbox/set update should fail on delegated mailboxes
     new b11b3f2  JAMES-3359 Mailbox/set update+destroy should fail on system mailboxes
     new 6bef18a  JAMES-3359 MailboxSetError "property" should use valueWrites
     new 22769e0  JAMES-3359 Mailbox/set update patch validation
     new e882d71  JAMES-3359 Remove TABs from MailboxSetMethodContract
     new 5477c5d  JAMES-3359 Mailbox/set update name tests for child mailbox
     new 8a87ede  JAMES-3358 Mailbox/set delete onDestroyRemoveEmails implementation
     new ccfbe53  JAMES-3357 Add sortOrder field in the created response for Mailbox/set method

The 14 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../apache/james/mailbox/MailboxManagerTest.java   |   95 ++
 .../james/mailbox/store/StoreRightManager.java     |   38 +-
 .../apache/james/mpt/host/JamesImapHostSystem.java |    2 +-
 .../cucumber/SetMailboxesMethodStepdefs.java       |   46 +-
 .../sharing/MailboxCreationAndSharing.feature      |    4 +
 .../contract/MailboxSetMethodContract.scala        | 1471 ++++++++++++++++++--
 .../org/apache/james/jmap/json/Serializer.scala    |   27 +-
 .../org/apache/james/jmap/mail/MailboxSet.scala    |   53 +-
 .../james/jmap/method/MailboxSetMethod.scala       |  143 +-
 9 files changed, 1708 insertions(+), 171 deletions(-)


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 01/14: JAMES-3361 JMAP Draft: sharee should not be able to modify mailbox rights

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit bb4029d48f7888a90e02b9ea30c7cd0c7554e4d9
Author: LanKhuat <dl...@linagora.com>
AuthorDate: Tue Aug 18 11:54:58 2020 +0700

    JAMES-3361 JMAP Draft: sharee should not be able to modify mailbox rights
---
 .../apache/james/mailbox/MailboxManagerTest.java   | 95 ++++++++++++++++++++++
 .../james/mailbox/store/StoreRightManager.java     | 38 ++++++---
 .../cucumber/SetMailboxesMethodStepdefs.java       | 46 ++++++++---
 .../sharing/MailboxCreationAndSharing.feature      |  4 +
 4 files changed, 161 insertions(+), 22 deletions(-)

diff --git a/mailbox/api/src/test/java/org/apache/james/mailbox/MailboxManagerTest.java b/mailbox/api/src/test/java/org/apache/james/mailbox/MailboxManagerTest.java
index 78d528e..04a4a68 100644
--- a/mailbox/api/src/test/java/org/apache/james/mailbox/MailboxManagerTest.java
+++ b/mailbox/api/src/test/java/org/apache/james/mailbox/MailboxManagerTest.java
@@ -2722,9 +2722,15 @@ public abstract class MailboxManagerTest<T extends MailboxManager> {
 
     @Nested
     class RightTests {
+
+        private MailboxSession session2;
+
         @BeforeEach
         void setUp() {
+            assumeTrue(mailboxManager.hasCapability(MailboxCapabilities.ACL));
+
             session = mailboxManager.createSystemSession(USER_1);
+            session2 = mailboxManager.createSystemSession(USER_2);
         }
 
         @Test
@@ -2796,5 +2802,94 @@ public abstract class MailboxManagerTest<T extends MailboxManager> {
             assertThatThrownBy(() -> mailboxManager.setRights(id, MailboxACL.EMPTY, session))
                 .isInstanceOf(MailboxNotFoundException.class);
         }
+
+        @Test
+        void setRightsByIdShouldThrowWhenNotOwner() throws Exception {
+            MailboxId id = mailboxManager.createMailbox(MailboxPath.forUser(USER_2, "mailbox"), session2).get();
+            mailboxManager.setRights(id,  MailboxACL.EMPTY.apply(MailboxACL.command()
+                .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                .rights(new MailboxACL.Rfc4314Rights(MailboxACL.Right.Lookup))
+                .asAddition()), session2);
+
+            assertThatThrownBy(() -> mailboxManager.setRights(id, MailboxACL.EMPTY.apply(
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition()), session))
+                .isInstanceOf(InsufficientRightsException.class);
+        }
+
+        @Test
+        void setRightsByPathShouldThrowWhenNotOwner() throws Exception {
+            MailboxPath mailboxPath = MailboxPath.forUser(USER_2, "mailbox");
+            mailboxManager.createMailbox(mailboxPath, session2).get();
+            mailboxManager.setRights(mailboxPath,  MailboxACL.EMPTY.apply(MailboxACL.command()
+                .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                .rights(new MailboxACL.Rfc4314Rights(MailboxACL.Right.Lookup))
+                .asAddition()), session2);
+
+            assertThatThrownBy(() -> mailboxManager.setRights(mailboxPath, MailboxACL.EMPTY.apply(
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition()), session))
+                .isInstanceOf(InsufficientRightsException.class);
+        }
+
+        @Test
+        void applyRightsCommandShouldThrowWhenNotOwner() throws Exception {
+            MailboxPath mailboxPath = MailboxPath.forUser(USER_2, "mailbox");
+            mailboxManager.createMailbox(mailboxPath, session2).get();
+            mailboxManager.setRights(mailboxPath,  MailboxACL.EMPTY.apply(MailboxACL.command()
+                .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                .rights(new MailboxACL.Rfc4314Rights(MailboxACL.Right.Lookup))
+                .asAddition()), session2);
+
+            assertThatThrownBy(() -> mailboxManager.applyRightsCommand(mailboxPath,
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition(), session))
+                .isInstanceOf(InsufficientRightsException.class);
+        }
+
+        @Test
+        void setRightsByIdShouldThrowWhenNoRights() throws Exception {
+            MailboxPath mailboxPath = MailboxPath.forUser(USER_2, "mailbox");
+            MailboxId mailboxId = mailboxManager.createMailbox(mailboxPath, session2).get();
+
+            assertThatThrownBy(() -> mailboxManager.setRights(mailboxId, MailboxACL.EMPTY.apply(
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition()), session))
+                .isInstanceOf(MailboxNotFoundException.class);
+        }
+
+        @Test
+        void setRightsByPathShouldThrowWhenNoRights() throws Exception {
+            MailboxPath mailboxPath = MailboxPath.forUser(USER_2, "mailbox");
+            mailboxManager.createMailbox(mailboxPath, session2).get();
+
+            assertThatThrownBy(() -> mailboxManager.setRights(mailboxPath, MailboxACL.EMPTY.apply(
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition()), session))
+                .isInstanceOf(MailboxNotFoundException.class);
+        }
+
+        @Test
+        void applyRightsCommandShouldThrowWhenNoRights() throws Exception {
+            MailboxPath mailboxPath = MailboxPath.forUser(USER_2, "mailbox");
+            mailboxManager.createMailbox(mailboxPath, session2).get();
+
+            assertThatThrownBy(() -> mailboxManager.applyRightsCommand(mailboxPath,
+                MailboxACL.command()
+                    .key(MailboxACL.EntryKey.createUserEntryKey(USER_1))
+                    .rights(MailboxACL.FULL_RIGHTS)
+                    .asAddition(), session))
+                .isInstanceOf(MailboxNotFoundException.class);
+        }
     }
 }
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreRightManager.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreRightManager.java
index ddb5e6d..afd7b90 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreRightManager.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreRightManager.java
@@ -38,6 +38,7 @@ import org.apache.james.mailbox.acl.MailboxACLResolver;
 import org.apache.james.mailbox.events.EventBus;
 import org.apache.james.mailbox.events.MailboxIdRegistrationKey;
 import org.apache.james.mailbox.exception.DifferentDomainException;
+import org.apache.james.mailbox.exception.InsufficientRightsException;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.exception.MailboxNotFoundException;
 import org.apache.james.mailbox.exception.UnsupportedRightException;
@@ -148,14 +149,18 @@ public class StoreRightManager implements RightManager {
         assertSharesBelongsToUserDomain(mailboxPath.getUser(), mailboxACLCommand);
         MailboxMapper mapper = mailboxSessionMapperFactory.getMailboxMapper(session);
         block(mapper.findMailboxByPath(mailboxPath)
-            .flatMap(mailbox -> mapper.updateACL(mailbox, mailboxACLCommand)
-                .flatMap(aclDiff -> eventBus.dispatch(EventFactory.aclUpdated()
-                        .randomEventId()
-                        .mailboxSession(session)
-                        .mailbox(mailbox)
-                        .aclDiff(aclDiff)
-                        .build(),
-                    new MailboxIdRegistrationKey(mailbox.getMailboxId())))));
+            .flatMap(Throwing.<Mailbox, Mono<Void>>function(mailbox -> {
+                assertHaveAccessTo(mailbox, session);
+
+                return mapper.updateACL(mailbox, mailboxACLCommand)
+                    .flatMap(aclDiff -> eventBus.dispatch(EventFactory.aclUpdated()
+                            .randomEventId()
+                            .mailboxSession(session)
+                            .mailbox(mailbox)
+                            .aclDiff(aclDiff)
+                            .build(),
+                        new MailboxIdRegistrationKey(mailbox.getMailboxId())));
+            }).sneakyThrow()));
     }
 
     private void assertSharesBelongsToUserDomain(Username user, ACLCommand mailboxACLCommand) throws DifferentDomainException {
@@ -208,10 +213,23 @@ public class StoreRightManager implements RightManager {
     @Override
     public void setRights(MailboxPath mailboxPath, MailboxACL mailboxACL, MailboxSession session) throws MailboxException {
         assertSharesBelongsToUserDomain(mailboxPath.getUser(), mailboxACL.getEntries());
-        MailboxMapper mapper = mailboxSessionMapperFactory.getMailboxMapper(session);
 
+        MailboxMapper mapper = mailboxSessionMapperFactory.getMailboxMapper(session);
         block(mapper.findMailboxByPath(mailboxPath)
-            .flatMap(mailbox -> setRights(mailboxACL, mapper, mailbox, session)));
+            .flatMap(Throwing.<Mailbox, Mono<Void>>function(mailbox -> {
+                assertHaveAccessTo(mailbox, session);
+                return setRights(mailboxACL, mapper, mailbox, session);
+            }).sneakyThrow()));
+    }
+
+    private void assertHaveAccessTo(Mailbox mailbox, MailboxSession session) throws InsufficientRightsException, MailboxNotFoundException {
+        if (!mailbox.generateAssociatedPath().belongsTo(session)) {
+            if (mailbox.getACL().getEntries().containsKey(EntryKey.createUserEntryKey(session.getUser()))) {
+                throw new InsufficientRightsException("Setting ACL is only permitted to the owner of the mailbox");
+            } else {
+                throw new MailboxNotFoundException(mailbox.getMailboxId());
+            }
+        }
     }
 
     @VisibleForTesting
diff --git a/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/java/org/apache/james/jmap/draft/methods/integration/cucumber/SetMailboxesMethodStepdefs.java b/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/java/org/apache/james/jmap/draft/methods/integration/cucumber/SetMailboxesMethodStepdefs.java
index 13547c0..b572d83 100644
--- a/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/java/org/apache/james/jmap/draft/methods/integration/cucumber/SetMailboxesMethodStepdefs.java
+++ b/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/java/org/apache/james/jmap/draft/methods/integration/cucumber/SetMailboxesMethodStepdefs.java
@@ -95,18 +95,40 @@ public class SetMailboxesMethodStepdefs {
         String mailboxId = mainStepdefs.getMailboxId(owner, mailboxName).serialize();
 
         String requestBody =
-                "[" +
-                    "  [ \"setMailboxes\"," +
-                    "    {" +
-                    "      \"update\": {" +
-                    "        \"" + mailboxId + "\" : {" +
-                    "          \"sharedWith\" : { \"" + shareTo + "\" : " + rightsAsString(rights) + " }" +
-                    "        }" +
-                    "      }" +
-                    "    }," +
-                    "    \"#0\"" +
-                    "  ]" +
-                    "]";
+            "[" +
+                "  [ \"setMailboxes\"," +
+                "    {" +
+                "      \"update\": {" +
+                "        \"" + mailboxId + "\" : {" +
+                "          \"sharedWith\" : { \"" + shareTo + "\" : " + rightsAsString(rights) + " }" +
+                "        }" +
+                "      }" +
+                "    }," +
+                "    \"#0\"" +
+                "  ]" +
+                "]";
+        httpClient.post(requestBody);
+    }
+
+    @Given("^\"([^\"]*)\" shares \"([^\"]*)\" delegated mailbox \"([^\"]*)\" with rights \"([^\"]*)\" with \"([^\"]*)\"$")
+    public void shareMailboxWithRight(String connectedUser, String owner,  String mailboxName, String rights, String shareTo) throws Throwable {
+        userStepdefs.connectUser(connectedUser);
+
+        String mailboxId = mainStepdefs.getMailboxId(owner, mailboxName).serialize();
+
+        String requestBody =
+            "[" +
+                "  [ \"setMailboxes\"," +
+                "    {" +
+                "      \"update\": {" +
+                "        \"" + mailboxId + "\" : {" +
+                "          \"sharedWith\" : { \"" + shareTo + "\" : " + rightsAsString(rights) + " }" +
+                "        }" +
+                "      }" +
+                "    }," +
+                "    \"#0\"" +
+                "  ]" +
+                "]";
         httpClient.post(requestBody);
     }
     
diff --git a/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/resources/cucumber/sharing/MailboxCreationAndSharing.feature b/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/resources/cucumber/sharing/MailboxCreationAndSharing.feature
index 9083333..0aa4081 100644
--- a/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/resources/cucumber/sharing/MailboxCreationAndSharing.feature
+++ b/server/protocols/jmap-draft-integration-testing/jmap-draft-integration-testing-common/src/test/resources/cucumber/sharing/MailboxCreationAndSharing.feature
@@ -26,6 +26,10 @@ Feature: Mailbox creation and sharing
     And "alice@domain.tld" has a mailbox "shared"
     And "alice@domain.tld" shares her mailbox "shared" with "bob@domain.tld" with "aeilrwt" rights
 
+  Scenario: A sharee should not be able to update shared mailbox rights
+    When "bob@domain.tld" shares "alice@domain.tld" delegated mailbox "shared" with rights "aeilrwt" with "bob@domain.tld"
+    Then mailbox "shared" owned by "alice@domain.tld" is not updated
+
   Scenario: A sharee should not be able to create a shared mailbox child
     Given "bob@domain.tld" creates mailbox "sharedChild" with creationId "c-01" in mailbox "shared" owned by "alice@domain.tld"
     When "alice@domain.tld" lists mailboxes


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 05/14: JAMES-3359 Mailbox/set update should handle Mailbox level errors

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 9fb1d3cdad231418155664450852011d571a67b3
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 14:38:03 2020 +0700

    JAMES-3359 Mailbox/set update should handle Mailbox level errors
---
 .../contract/MailboxSetMethodContract.scala        | 114 ++++++++++++++++++++-
 .../james/jmap/method/MailboxSetMethod.scala       |   2 +
 2 files changed, 114 insertions(+), 2 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 180a198..fefaebc 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2165,9 +2165,119 @@ trait MailboxSetMethodContract {
          |}""".stripMargin)
   }
 
+  @Test
+  def updateShouldFailWhenTargetMailboxAlreadyExist(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "newName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/name": "newName"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidArguments",
+         |					"description": "Mailbox with name=#private:bob@domain.tld:newName already exists.",
+         |					"properties": {
+         |						"value": ["/name"]
+         |					}
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def updateShouldFailWhenMailboxNameIsTooLong(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/name": "${"a".repeat(201)}"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidArguments",
+         |					"description": "Mailbox name exceeds maximum size of 200 characters",
+         |					"properties": {
+         |						"value": ["/name"]
+         |					}
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
   // TODO invalid path handling (unknown property, invalid name)
-  // TODO mailbox already exists, too long name
-  // TODO disable destroy / rename of sustem mailbox
+  // TODO disable destroy / rename of system mailbox
   // TODO test that renames keeps subscriptions
   // TODO renaming delegated mailboxes is not allowed
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index e774a0b..8122a42 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -118,6 +118,8 @@ sealed trait UpdateResult
 case class UpdateSuccess(mailboxId: MailboxId) extends UpdateResult
 case class UpdateFailure(mailboxId: UnparsedMailboxId, exception: Throwable) extends UpdateResult {
   def asMailboxSetError: MailboxSetError = exception match {
+    case e: MailboxNameException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
+    case e: MailboxExistsException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 03/14: JAMES-3359 Mailbox/set update name support

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 0b2d84224c31fb9ebc57eb7bec7193764a17b6d1
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 12:38:14 2020 +0700

    JAMES-3359 Mailbox/set update name support
    
    This only comprises basic support, error handling and advanced features needs to be implemented.
    
    TODO update using creation ids
    TODO invalid path handling (unknown property, invalid name)
    TODO mailbox already exists, too long name
    TODO disable destroy / rename of sustem mailbox
    TODO test that renames keeps subscriptions
    TODO renaming delegated mailboxes is not allowed
---
 .../contract/MailboxSetMethodContract.scala        | 90 ++++++++++++++++++++
 .../org/apache/james/jmap/json/Serializer.scala    |  2 +-
 .../org/apache/james/jmap/mail/MailboxSet.scala    | 31 ++++++-
 .../james/jmap/method/MailboxSetMethod.scala       | 95 ++++++++++++++++++++--
 4 files changed, 208 insertions(+), 10 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index f8d8a96..b1853b9 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -1993,4 +1993,94 @@ trait MailboxSetMethodContract {
          |	]
          |}""".stripMargin)
   }
+
+  @Test
+  def updateShouldRenameMailboxes(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox1"))
+
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |      ["Mailbox/set",
+        |          {
+        |               "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |               "update": {
+        |                 "${mailboxId.serialize()}" : {
+        |                   "/name": "newName"
+        |                 }
+        |               }
+        |          },
+        |   "c2"],
+        |      ["Mailbox/get",
+        |         {
+        |           "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |           "ids": ["${mailboxId.serialize()}"]
+        |          },
+        |       "c2"]
+        |   ]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"updated": {
+         |				"1": {}
+         |			}
+         |		}, "c2"],
+         |		["Mailbox/get", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"state": "000001",
+         |			"list": [{
+         |				"id": "1",
+         |				"name": "newName",
+         |				"sortOrder": 1000,
+         |				"totalEmails": 0,
+         |				"unreadEmails": 0,
+         |				"totalThreads": 0,
+         |				"unreadThreads": 0,
+         |				"myRights": {
+         |					"mayReadItems": true,
+         |					"mayAddItems": true,
+         |					"mayRemoveItems": true,
+         |					"maySetSeen": true,
+         |					"maySetKeywords": true,
+         |					"mayCreateChild": true,
+         |					"mayRename": true,
+         |					"mayDelete": true,
+         |					"maySubmit": true
+         |				},
+         |				"isSubscribed": false
+         |			}],
+         |			"notFound": []
+         |		}, "c2"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  // TODO update using creation ids
+  // TODO invalid path handling (unknown property, invalid name)
+  // TODO mailbox already exists, too long name
+  // TODO disable destroy / rename of sustem mailbox
+  // TODO test that renames keeps subscriptions
+  // TODO renaming delegated mailboxes is not allowed
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
index 131bb41..b655527 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
@@ -276,7 +276,7 @@ class Serializer @Inject() (mailboxIdFactory: MailboxId.Factory) {
 
   private implicit val mailboxSetCreationResponseWrites: Writes[MailboxCreationResponse] = Json.writes[MailboxCreationResponse]
 
-  private implicit val mailboxSetUpdateResponseWrites: Writes[MailboxUpdateResponse] = Json.writes[MailboxUpdateResponse]
+  private implicit val mailboxSetUpdateResponseWrites: Writes[MailboxUpdateResponse] = Json.valueWrites[MailboxUpdateResponse]
 
   private implicit val propertiesWrites: Writes[Properties] = Json.writes[Properties]
 
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
index f50e5ff..51821dd 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
@@ -29,7 +29,7 @@ import org.apache.james.jmap.model.AccountId
 import org.apache.james.jmap.model.State.State
 import org.apache.james.mailbox.Role
 import org.apache.james.mailbox.model.MailboxId
-import play.api.libs.json.JsObject
+import play.api.libs.json.{JsObject, JsString, JsValue}
 
 case class MailboxSetRequest(accountId: AccountId,
                              ifInState: Option[State],
@@ -55,7 +55,14 @@ case class MailboxCreationRequest(name: MailboxName,
                                   isSubscribed: Option[IsSubscribed],
                                   rights: Option[Rights])
 
-case class MailboxPatchObject(value: Map[String, JsObject])
+case class MailboxPatchObject(value: Map[String, JsValue]) {
+  def updates: Iterable[Either[PatchUpdateValidationException, Update]] = value.map({
+    case (property, newValue) => property match {
+      case "/name" => NameUpdate.parse(newValue)
+      case property => Left(UnsupportedPropertyUpdated(property))
+    }
+  })
+}
 
 case class MailboxSetResponse(accountId: AccountId,
                               oldState: Option[State],
@@ -98,4 +105,22 @@ case class MailboxCreationResponse(id: MailboxId,
                                    isSubscribed: IsSubscribed
                                   )
 
-case class MailboxUpdateResponse(value: JsObject)
\ No newline at end of file
+object MailboxSetResponse {
+  def empty: MailboxUpdateResponse = MailboxUpdateResponse(JsObject(Map[String, JsValue]()))
+}
+
+case class MailboxUpdateResponse(value: JsObject)
+
+object NameUpdate {
+  def parse(newValue: JsValue): Either[PatchUpdateValidationException, Update] = newValue match {
+    case JsString(newName) => scala.Right(NameUpdate(newName))
+    case _ => Left(InvalidUpdate("name", "Expectint a JSON string as an argument"))
+  }
+}
+
+sealed trait Update
+case class NameUpdate(newName: String) extends Update
+
+class PatchUpdateValidationException() extends IllegalArgumentException
+case class UnsupportedPropertyUpdated(property: String) extends PatchUpdateValidationException
+case class InvalidUpdate(property: String, cause: String) extends PatchUpdateValidationException
\ No newline at end of file
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 6ac4d5a..81fc67d 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -23,20 +23,23 @@ import eu.timepit.refined.auto._
 import javax.inject.Inject
 import org.apache.james.jmap.json.Serializer
 import org.apache.james.jmap.mail.MailboxSetRequest.{MailboxCreationId, UnparsedMailboxId}
-import org.apache.james.jmap.mail.{IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, Properties, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads}
+import org.apache.james.jmap.mail.{IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads}
 import org.apache.james.jmap.model.CapabilityIdentifier.CapabilityIdentifier
 import org.apache.james.jmap.model.Invocation.{Arguments, MethodName}
 import org.apache.james.jmap.model.{ClientId, Id, Invocation, ServerId, State}
 import org.apache.james.jmap.routes.ProcessingContext
+import org.apache.james.mailbox.MailboxManager.RenameOption
 import org.apache.james.mailbox.exception.{InsufficientRightsException, MailboxExistsException, MailboxNameException, MailboxNotFoundException}
 import org.apache.james.mailbox.model.{FetchGroup, MailboxId, MailboxPath, MessageRange}
-import org.apache.james.mailbox.{MailboxManager, MailboxSession, SubscriptionManager}
+import org.apache.james.mailbox.{MailboxManager, MailboxSession, MessageManager, SubscriptionManager}
 import org.apache.james.metrics.api.MetricFactory
 import org.reactivestreams.Publisher
 import play.api.libs.json._
 import reactor.core.scala.publisher.{SFlux, SMono}
 import reactor.core.scheduler.Schedulers
 
+import scala.jdk.CollectionConverters._
+
 case class MailboxHasMailException(mailboxId: MailboxId) extends Exception
 case class MailboxHasChildException(mailboxId: MailboxId) extends Exception
 case class MailboxCreationParseException(mailboxSetError: MailboxSetError) extends Exception
@@ -111,6 +114,25 @@ case class DeletionResults(results: Seq[DeletionResult]) {
     .toMap
 }
 
+sealed trait UpdateResult
+case class UpdateSuccess(mailboxId: MailboxId) extends UpdateResult
+case class UpdateFailure(mailboxId: MailboxId, exception: Throwable) extends UpdateResult {
+  def asMailboxSetError: MailboxSetError = exception match {
+    case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
+  }
+}
+case class UpdateResults(results: Seq[UpdateResult]) {
+  def updated: Map[MailboxId, MailboxUpdateResponse] =
+    results.flatMap(result => result match {
+      case success: UpdateSuccess => Some((success.mailboxId, MailboxSetResponse.empty))
+      case _ => None
+    }).toMap
+  def notUpdated: Map[MailboxId, MailboxSetError] = results.flatMap(result => result match {
+    case failure: UpdateFailure => Some(failure.mailboxId, failure.asMailboxSetError)
+    case _ => None
+  }).toMap
+}
+
 class MailboxSetMethod @Inject()(serializer: Serializer,
                                  mailboxManager: MailboxManager,
                                  subscriptionManager: SubscriptionManager,
@@ -125,10 +147,70 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
           for {
             creationResults <- createMailboxes(mailboxSession, mailboxSetRequest, processingContext)
             deletionResults <- deleteMailboxes(mailboxSession, mailboxSetRequest, processingContext)
-          } yield createResponse(invocation, mailboxSetRequest, creationResults, deletionResults)
+            updateResults <- updateMailboxes(mailboxSession, mailboxSetRequest, processingContext)
+          } yield createResponse(invocation, mailboxSetRequest, creationResults, deletionResults, updateResults)
         }))
   }
 
+  private def updateMailboxes(mailboxSession: MailboxSession,
+                              mailboxSetRequest: MailboxSetRequest,
+                              processingContext: ProcessingContext): SMono[UpdateResults] = {
+    SFlux.fromIterable(mailboxSetRequest.update.getOrElse(Seq()))
+      .flatMap({
+        case (mailboxId: MailboxId, patch: MailboxPatchObject) => updateMailbox(mailboxSession, mailboxId, patch)
+          .onErrorResume(e => SMono.just(UpdateFailure(mailboxId, e)))
+      })
+      .collectSeq()
+      .map(UpdateResults)
+  }
+
+  private def updateMailbox(mailboxSession: MailboxSession,
+                            maiboxId: MailboxId,
+                            patch: MailboxPatchObject): SMono[UpdateResult] = {
+    val maybeParseException: Option[PatchUpdateValidationException] = patch.updates
+      .flatMap(x => x match {
+        case Left(e) => Some(e)
+        case _ => None
+      }).headOption
+
+    val maybeNameUpdate: Option[NameUpdate] = patch.updates
+      .flatMap(x => x match {
+        case Right(NameUpdate(newName)) => Some(NameUpdate(newName))
+        case _ => None
+      }).headOption
+
+    def renameMailbox: SMono[UpdateResult] = updateMailboxPath(maiboxId, maybeNameUpdate, mailboxSession)
+
+    maybeParseException.map(e => SMono.just[UpdateResult](UpdateFailure(maiboxId, e)))
+      .getOrElse(renameMailbox)
+  }
+
+  private def updateMailboxPath(maiboxId: MailboxId, maybeNameUpdate: Option[NameUpdate], mailboxSession: MailboxSession): SMono[UpdateResult] = {
+    maybeNameUpdate.map(nameUpdate => {
+      SMono.fromCallable(() => {
+        val mailbox = mailboxManager.getMailbox(maiboxId, mailboxSession)
+        mailboxManager.renameMailbox(maiboxId,
+          computeMailboxPath(mailbox, nameUpdate, mailboxSession),
+          RenameOption.RENAME_SUBSCRIPTIONS,
+          mailboxSession)
+      }).`then`(SMono.just[UpdateResult](UpdateSuccess(maiboxId)))
+        .subscribeOn(Schedulers.elastic())
+    })
+      // No updated properties passed. Noop.
+      .getOrElse(SMono.just[UpdateResult](UpdateSuccess(maiboxId)))
+  }
+
+  private def computeMailboxPath(mailbox: MessageManager, nameUpdate: NameUpdate, mailboxSession: MailboxSession): MailboxPath = {
+    val originalPath: MailboxPath = mailbox.getMailboxPath
+    val maybeParentPath: Option[MailboxPath] = originalPath.getHierarchyLevels(mailboxSession.getPathDelimiter)
+      .asScala
+      .reverse
+      .drop(1)
+      .headOption
+    maybeParentPath.map(_.child(nameUpdate.newName, mailboxSession.getPathDelimiter))
+      .getOrElse(MailboxPath.forUser(mailboxSession.getUser, nameUpdate.newName))
+  }
+
   private def deleteMailboxes(mailboxSession: MailboxSession, mailboxSetRequest: MailboxSetRequest, processingContext: ProcessingContext): SMono[DeletionResults] = {
     SFlux.fromIterable(mailboxSetRequest.destroy.getOrElse(Seq()))
       .flatMap(id => delete(mailboxSession, processingContext, id)
@@ -257,7 +339,8 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
   private def createResponse(invocation: Invocation,
                              mailboxSetRequest: MailboxSetRequest,
                              creationResults: CreationResults,
-                             deletionResults: DeletionResults): Invocation = {
+                             deletionResults: DeletionResults,
+                             updateResults: UpdateResults): Invocation = {
     val response = MailboxSetResponse(
       mailboxSetRequest.accountId,
       oldState = None,
@@ -265,8 +348,8 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
       destroyed = Some(deletionResults.destroyed).filter(_.nonEmpty),
       created = Some(creationResults.retrieveCreated).filter(_.nonEmpty),
       notCreated = Some(creationResults.retrieveErrors).filter(_.nonEmpty),
-      updated = None,
-      notUpdated = None,
+      updated = Some(updateResults.updated).filter(_.nonEmpty),
+      notUpdated = Some(updateResults.notUpdated).filter(_.nonEmpty),
       notDestroyed = Some(deletionResults.retrieveErrors).filter(_.nonEmpty))
     
     Invocation(methodName,


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 13/14: JAMES-3358 Mailbox/set delete onDestroyRemoveEmails implementation

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 8a87ede382974519610e1a194dae440a4f12022e
Author: LanKhuat <dl...@linagora.com>
AuthorDate: Mon Aug 17 11:25:50 2020 +0700

    JAMES-3358 Mailbox/set delete onDestroyRemoveEmails implementation
---
 .../contract/MailboxSetMethodContract.scala        | 303 ++++++++++++++++++++-
 .../org/apache/james/jmap/json/Serializer.scala    |   2 +-
 .../james/jmap/method/MailboxSetMethod.scala       |  29 +-
 3 files changed, 322 insertions(+), 12 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 9f7eccc..582ce3a 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -29,6 +29,7 @@ import net.javacrumbs.jsonunit.core.Option
 import net.javacrumbs.jsonunit.core.internal.Options
 import org.apache.http.HttpStatus.SC_OK
 import org.apache.james.GuiceJamesServer
+import org.apache.james.jmap.draft.MessageIdProbe
 import org.apache.james.jmap.http.UserCredential
 import org.apache.james.jmap.rfc8621.contract.Fixture.{ACCEPT_RFC8621_VERSION_HEADER, ANDRE, BOB, BOB_PASSWORD, DOMAIN, authScheme, baseRequestSpecBuilder}
 import org.apache.james.mailbox.MessageManager.AppendCommand
@@ -37,8 +38,8 @@ import org.apache.james.mailbox.model.{MailboxACL, MailboxId, MailboxPath}
 import org.apache.james.mime4j.dom.Message
 import org.apache.james.modules.{ACLProbeImpl, MailboxProbeImpl}
 import org.apache.james.utils.DataProbeImpl
-import org.assertj.core.api.Assertions
 import org.assertj.core.api.Assertions.assertThat
+import org.assertj.core.api.{Assertions, SoftAssertions}
 import org.hamcrest.Matchers.{equalTo, hasSize}
 import org.junit.jupiter.api.{BeforeEach, Disabled, Test}
 
@@ -2911,4 +2912,304 @@ trait MailboxSetMethodContract {
          |  ]
          |}""".stripMargin)
   }
+
+  @Test
+  def deleteShouldSucceedWhenOnDestroyRemoveEmails(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox"))
+    val message: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail", StandardCharsets.UTF_8)
+      .build
+
+    server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message))
+
+    val request =
+      s"""
+         |{
+         |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+         |   "methodCalls": [
+         |       [
+         |           "Mailbox/set",
+         |           {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "destroy": ["${mailboxId.serialize}"],
+         |                "onDestroyRemoveEmails": true
+         |           },
+         |    "c1"
+         |       ]
+         |   ]
+         |}
+         |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [[
+         |    "Mailbox/set",
+         |    {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "destroyed": ["${mailboxId.serialize}"]
+         |    },
+         |    "c1"]]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def deleteShouldRemoveMailboxWhenOnDestroyRemoveEmails(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox"))
+    val message: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail", StandardCharsets.UTF_8)
+      .build
+
+    server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message))
+
+    val request =
+      s"""
+         |{
+         |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+         |   "methodCalls": [
+         |       [
+         |           "Mailbox/set",
+         |           {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "destroy": ["${mailboxId.serialize}"],
+         |                "onDestroyRemoveEmails": true
+         |           },
+         |    "c1"
+         |       ]
+         |   ]
+         |}
+         |""".stripMargin
+
+    `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(s"""{
+               |  "using": [
+               |    "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail"],
+               |  "methodCalls": [[
+               |      "Mailbox/get",
+               |      {
+               |        "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+               |        "ids": ["${mailboxId.serialize()}"]
+               |      },
+               |      "c1"]]
+               |}""".stripMargin)
+    .when
+      .post
+    .`then`()
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |    "sessionState": "75128aab4b1b",
+         |    "methodResponses": [
+         |        [
+         |            "Mailbox/get",
+         |            {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "state": "000001",
+         |                "list": [
+         |
+         |                ],
+         |                "notFound": [
+         |                    "${mailboxId.serialize()}"
+         |                ]
+         |            },
+         |            "c1"
+         |        ]
+         |    ]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def deleteShouldRemoveMessagesWhenOnDestroyRemoveEmails(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox"))
+    val message1: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail1", StandardCharsets.UTF_8)
+      .build
+
+    val message2: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail2", StandardCharsets.UTF_8)
+      .build
+
+    val messageId1 = server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message1))
+    val messageId2 = server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message2))
+
+    val request =
+      s"""
+         |{
+         |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+         |   "methodCalls": [
+         |       [
+         |           "Mailbox/set",
+         |           {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "destroy": ["${mailboxId.serialize}"],
+         |                "onDestroyRemoveEmails": true
+         |           },
+         |    "c1"
+         |       ]
+         |   ]
+         |}
+         |""".stripMargin
+
+    `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+
+    //Should be replaced with JMAP message query when it is available
+    SoftAssertions.assertSoftly(softly => {
+      softly.assertThat(server.getProbe(classOf[MessageIdProbe]).getMessages(messageId1.getMessageId, BOB)).isEmpty()
+      softly.assertThat(server.getProbe(classOf[MessageIdProbe]).getMessages(messageId2.getMessageId, BOB)).isEmpty()
+    })
+  }
+
+  @Test
+  def deleteShouldFailWhenMailboxIsNotEmptyAndOnDestroyRemoveEmailsIsFalse(server: GuiceJamesServer): Unit = {
+    val message: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail", StandardCharsets.UTF_8)
+      .build
+
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox"))
+    server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message))
+
+    val request =
+      s"""
+         |{
+         |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+         |   "methodCalls": [
+         |       [
+         |           "Mailbox/set",
+         |           {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "destroy": ["${mailboxId.serialize()}"],
+         |                "onDestroyRemoveEmails": false
+         |           },
+         |    "c1"
+         |       ]
+         |   ]
+         |}
+         |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [[
+         |    "Mailbox/set",
+         |    {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notDestroyed": {
+         |        "${mailboxId.serialize()}": {
+         |          "type": "mailboxHasEmail",
+         |          "description": "${mailboxId.serialize()} is not empty"
+         |        }
+         |      }
+         |    },
+         |    "c1"]]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def deleteShouldNotRemoveMessageWhenMailboxIsNotEmptyAndOnDestroyRemoveEmailsIsFalse(server: GuiceJamesServer): Unit = {
+    val message: Message = Message.Builder
+      .of
+      .setSubject("test")
+      .setBody("testmail", StandardCharsets.UTF_8)
+      .build
+
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "mailbox"))
+    val messageId = server.getProbe(classOf[MailboxProbeImpl])
+      .appendMessage(BOB.asString, MailboxPath.forUser(BOB, "mailbox"), AppendCommand.from(message))
+
+    val request =
+      s"""
+         |{
+         |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+         |   "methodCalls": [
+         |       [
+         |           "Mailbox/set",
+         |           {
+         |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |                "destroy": ["${mailboxId.serialize()}"],
+         |                "onDestroyRemoveEmails": false
+         |           },
+         |    "c1"
+         |       ]
+         |   ]
+         |}
+         |""".stripMargin
+
+    `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+
+    //Should be replaced with JMAP message query when it is available
+    assertThat(server.getProbe(classOf[MessageIdProbe]).getMessages(messageId.getMessageId, BOB)).isNotEmpty
+  }
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
index eae6a9d..212e840 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
@@ -237,7 +237,7 @@ class Serializer @Inject() (mailboxIdFactory: MailboxId.Factory) {
   private implicit val mailboxGetRequest: Reads[MailboxGetRequest] = Json.reads[MailboxGetRequest]
 
 
-  private implicit val mailboxRemoveEmailsOnDestroy: Reads[RemoveEmailsOnDestroy] = Json.reads[RemoveEmailsOnDestroy]
+  private implicit val mailboxRemoveEmailsOnDestroy: Reads[RemoveEmailsOnDestroy] = Json.valueFormat[RemoveEmailsOnDestroy]
   implicit val mailboxCreationRequest: Reads[MailboxCreationRequest] = Json.reads[MailboxCreationRequest]
   private implicit val mailboxPatchObject: Reads[MailboxPatchObject] = Json.valueReads[MailboxPatchObject]
 
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 2d10905..2077181 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -23,7 +23,7 @@ import eu.timepit.refined.auto._
 import javax.inject.Inject
 import org.apache.james.jmap.json.Serializer
 import org.apache.james.jmap.mail.MailboxSetRequest.{MailboxCreationId, UnparsedMailboxId}
-import org.apache.james.jmap.mail.{InvalidPropertyException, InvalidUpdateException, IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads, UnsupportedPropertyUpdatedException}
+import org.apache.james.jmap.mail.{InvalidPropertyException, InvalidUpdateException, IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, RemoveEmailsOnDestroy, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads, UnsupportedPropertyUpdatedException}
 import org.apache.james.jmap.model.CapabilityIdentifier.CapabilityIdentifier
 import org.apache.james.jmap.model.Invocation.{Arguments, MethodName}
 import org.apache.james.jmap.model.{ClientId, Id, Invocation, ServerId, State}
@@ -228,34 +228,43 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
 
   private def deleteMailboxes(mailboxSession: MailboxSession, mailboxSetRequest: MailboxSetRequest, processingContext: ProcessingContext): SMono[DeletionResults] = {
     SFlux.fromIterable(mailboxSetRequest.destroy.getOrElse(Seq()))
-      .flatMap(id => delete(mailboxSession, processingContext, id)
+      .flatMap(id => delete(mailboxSession, processingContext, id, mailboxSetRequest.onDestroyRemoveEmails.getOrElse(RemoveEmailsOnDestroy(false)))
         .onErrorRecover(e => DeletionFailure(id, e)))
       .collectSeq()
       .map(DeletionResults)
   }
 
-  private def delete(mailboxSession: MailboxSession, processingContext: ProcessingContext, id: UnparsedMailboxId): SMono[DeletionResult] = {
+  private def delete(mailboxSession: MailboxSession, processingContext: ProcessingContext, id: UnparsedMailboxId, onDestroy: RemoveEmailsOnDestroy): SMono[DeletionResult] = {
     processingContext.resolveMailboxId(id, mailboxIdFactory) match {
-      case Right(mailboxId) => SMono.fromCallable(() => delete(mailboxSession, mailboxId))
+      case Right(mailboxId) => SMono.fromCallable(() => delete(mailboxSession, mailboxId, onDestroy))
         .subscribeOn(Schedulers.elastic())
         .`then`(SMono.just[DeletionResult](DeletionSuccess(mailboxId)))
       case Left(e) => SMono.raiseError(e)
     }
   }
 
-  private def delete(mailboxSession: MailboxSession, id: MailboxId): Unit = {
+  private def delete(mailboxSession: MailboxSession, id: MailboxId, onDestroy: RemoveEmailsOnDestroy): Unit = {
     val mailbox = mailboxManager.getMailbox(id, mailboxSession)
+
     if (isASystemMailbox(mailbox)) {
       throw SystemMailboxChangeException(id)
     }
-    if (mailbox.getMessages(MessageRange.all(), FetchGroup.MINIMAL, mailboxSession).hasNext) {
-      throw MailboxHasMailException(id)
-    }
+
     if (mailboxManager.hasChildren(mailbox.getMailboxPath, mailboxSession)) {
       throw MailboxHasChildException(id)
     }
-    val deletedMailbox = mailboxManager.deleteMailbox(id, mailboxSession)
-    subscriptionManager.unsubscribe(mailboxSession, deletedMailbox.getName)
+
+    if (onDestroy.value) {
+      val deletedMailbox = mailboxManager.deleteMailbox(id, mailboxSession)
+      subscriptionManager.unsubscribe(mailboxSession, deletedMailbox.getName)
+    } else {
+      if (mailbox.getMessages(MessageRange.all(), FetchGroup.MINIMAL, mailboxSession).hasNext) {
+        throw MailboxHasMailException(id)
+      }
+
+      val deletedMailbox = mailboxManager.deleteMailbox(id, mailboxSession)
+      subscriptionManager.unsubscribe(mailboxSession, deletedMailbox.getName)
+    }
   }
 
   private def isASystemMailbox(mailbox: MessageManager): Boolean = Role.from(mailbox.getMailboxPath.getName).isPresent


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 09/14: JAMES-3359 MailboxSetError "property" should use valueWrites

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 6bef18ade4dc8bb48450f102a3cf69d7bb243b16
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 15:19:18 2020 +0700

    JAMES-3359 MailboxSetError "property" should use valueWrites
---
 .../rfc8621/contract/MailboxSetMethodContract.scala    | 18 +++++++-----------
 .../scala/org/apache/james/jmap/json/Serializer.scala  |  2 +-
 2 files changed, 8 insertions(+), 12 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index cebae49..8d682ca 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -993,7 +993,7 @@ trait MailboxSetMethodContract {
          |        "C42": {
          |          "type": "invalidArguments",
          |          "description": "${mailboxId.serialize()} can not be found",
-         |          "properties":{"value":["parentId"]}
+         |          "properties":["parentId"]
          |        }
          |      }
          |    },
@@ -1050,7 +1050,7 @@ trait MailboxSetMethodContract {
          |        "C42": {
          |          "type": "invalidArguments",
          |          "description": "Mailbox with name=#private:bob@domain.tld:mailbox already exists.",
-         |          "properties":{"value":["name"]}
+         |          "properties":["name"]
          |        }
          |      }
          |    },
@@ -1106,7 +1106,7 @@ trait MailboxSetMethodContract {
          |        "C42": {
          |          "type": "invalidArguments",
          |          "description": "Mailbox name exceeds maximum size of 200 characters",
-         |          "properties":{"value":["name"]}
+         |          "properties":["name"]
          |        }
          |      }
          |    },
@@ -1168,7 +1168,7 @@ trait MailboxSetMethodContract {
          |        "C42": {
          |          "type": "forbidden",
          |          "description": "Insufficient rights",
-         |          "properties":{"value":["parentId"]}
+         |          "properties":["parentId"]
          |        }
          |      }
          |    },
@@ -2253,9 +2253,7 @@ trait MailboxSetMethodContract {
          |				"${mailboxId1.serialize()}": {
          |					"type": "invalidArguments",
          |					"description": "Mailbox with name=#private:bob@domain.tld:newName already exists.",
-         |					"properties": {
-         |						"value": ["/name"]
-         |					}
+         |					"properties": ["/name"]
          |				}
          |			}
          |		}, "c1"]
@@ -2308,9 +2306,7 @@ trait MailboxSetMethodContract {
          |				"${mailboxId1.serialize()}": {
          |					"type": "invalidArguments",
          |					"description": "Mailbox name exceeds maximum size of 200 characters",
-         |					"properties": {
-         |						"value": ["/name"]
-         |					}
+         |					"properties": ["/name"]
          |				}
          |			}
          |		}, "c1"]
@@ -2471,7 +2467,7 @@ trait MailboxSetMethodContract {
          |				"${mailboxId.serialize()}": {
          |					"type": "invalidArguments",
          |					"description": "Invalid change to a system mailbox",
-         |          "properties":{"value":["/name"]}
+         |          "properties":["/name"]
          |				}
          |			}
          |		}, "c1"]
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
index a813bf3..eae6a9d 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
@@ -293,7 +293,7 @@ class Serializer @Inject() (mailboxIdFactory: MailboxId.Factory) {
 
   private implicit val mailboxSetUpdateResponseWrites: Writes[MailboxUpdateResponse] = Json.valueWrites[MailboxUpdateResponse]
 
-  private implicit val propertiesWrites: Writes[Properties] = Json.writes[Properties]
+  private implicit val propertiesWrites: Writes[Properties] = Json.valueWrites[Properties]
 
   private implicit val setErrorDescriptionWrites: Writes[SetErrorDescription] = Json.valueWrites[SetErrorDescription]
 


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 07/14: JAMES-3359 Mailbox/set update should fail on delegated mailboxes

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit b4427970abd7ab64a672fcfd797da50b40d3572d
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 14:55:57 2020 +0700

    JAMES-3359 Mailbox/set update should fail on delegated mailboxes
---
 .../contract/MailboxSetMethodContract.scala        | 108 ++++++++++++++++++++-
 .../james/jmap/method/MailboxSetMethod.scala       |   1 +
 2 files changed, 108 insertions(+), 1 deletion(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index d57b247..3436f8f 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2318,7 +2318,113 @@ trait MailboxSetMethodContract {
          |}""".stripMargin)
   }
 
+  @Test
+  def updateShouldFailWhenMailboxNotFound(): Unit = {
+    val mailboxId1: MailboxId = randomMailboxId
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/name": "newName"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "notFound",
+         |					"description": "${mailboxId1.serialize()} can not be found"
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def updateShouldNotRenameDelegtedMailboxes(server: GuiceJamesServer): Unit = {
+    val path = MailboxPath.forUser(ANDRE, "previousName")
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(path)
+    server.getProbe(classOf[ACLProbeImpl])
+      .replaceRights(path, BOB.asString, MailboxACL.FULL_RIGHTS)
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/name": "newName"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "notFound",
+         |					"description": "#private:andre@domain.tld:previousName"
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
   // TODO invalid path handling (unknown property, invalid name)
   // TODO disable destroy / rename of system mailbox
-  // TODO renaming delegated mailboxes is not allowed
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 8122a42..ea79a9c 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -118,6 +118,7 @@ sealed trait UpdateResult
 case class UpdateSuccess(mailboxId: MailboxId) extends UpdateResult
 case class UpdateFailure(mailboxId: UnparsedMailboxId, exception: Throwable) extends UpdateResult {
   def asMailboxSetError: MailboxSetError = exception match {
+    case e: MailboxNotFoundException => MailboxSetError.notFound(Some(SetErrorDescription(e.getMessage)))
     case e: MailboxNameException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
     case e: MailboxExistsException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 12/14: JAMES-3359 Mailbox/set update name tests for child mailbox

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 5477c5df2c9d61f243f87e8703450cf3462edb96
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 16:12:51 2020 +0700

    JAMES-3359 Mailbox/set update name tests for child mailbox
---
 .../contract/MailboxSetMethodContract.scala        | 176 +++++++++++++++++++++
 1 file changed, 176 insertions(+)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index fdf59da..9f7eccc 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2735,4 +2735,180 @@ trait MailboxSetMethodContract {
          |  ]
          |}""".stripMargin)
   }
+
+  @Test
+  def nameUpdatesShouldNotAffectParentId(server: GuiceJamesServer): Unit = {
+    val parentId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "parent"))
+    val childId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "parent.oldChild"))
+    val request=
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       [
+        |           "Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${childId.serialize()}": {
+        |                      "/name": "newChild"
+        |                    }
+        |                }
+        |           },
+        |    "c1"
+        |       ],
+        |       ["Mailbox/get",
+        |         {
+        |           "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |           "ids": ["${childId.serialize()}"]
+        |          },
+        |       "c2"]
+        |   ]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "updated": {
+         |        "${childId.serialize()}": {}
+         |      }
+         |    }, "c1"],
+         |    ["Mailbox/get", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "state": "000001",
+         |      "list": [{
+         |        "id": "${childId.serialize()}",
+         |        "name": "newChild",
+         |        "parentId": "${parentId.serialize()}",
+         |        "sortOrder": 1000,
+         |        "totalEmails": 0,
+         |        "unreadEmails": 0,
+         |        "totalThreads": 0,
+         |        "unreadThreads": 0,
+         |        "myRights": {
+         |          "mayReadItems": true,
+         |          "mayAddItems": true,
+         |          "mayRemoveItems": true,
+         |          "maySetSeen": true,
+         |          "maySetKeywords": true,
+         |          "mayCreateChild": true,
+         |          "mayRename": true,
+         |          "mayDelete": true,
+         |          "maySubmit": true
+         |        },
+         |        "isSubscribed": false
+         |      }],
+         |      "notFound": []
+         |    }, "c2"]
+         |  ]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def parentRenameShouldNotAffectChild(server: GuiceJamesServer): Unit = {
+    val parentId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "parent"))
+    val childId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "parent.child"))
+    val request=
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       [
+        |           "Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${parentId.serialize()}": {
+        |                      "/name": "newParent"
+        |                    }
+        |                }
+        |           },
+        |    "c1"
+        |       ],
+        |       ["Mailbox/get",
+        |         {
+        |           "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |           "ids": ["${childId.serialize()}"]
+        |          },
+        |       "c2"]
+        |   ]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "updated": {
+         |        "${parentId.serialize()}": {}
+         |      }
+         |    }, "c1"],
+         |    ["Mailbox/get", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "state": "000001",
+         |      "list": [{
+         |        "id": "${childId.serialize()}",
+         |        "name": "child",
+         |        "parentId": "${parentId.serialize()}",
+         |        "sortOrder": 1000,
+         |        "totalEmails": 0,
+         |        "unreadEmails": 0,
+         |        "totalThreads": 0,
+         |        "unreadThreads": 0,
+         |        "myRights": {
+         |          "mayReadItems": true,
+         |          "mayAddItems": true,
+         |          "mayRemoveItems": true,
+         |          "maySetSeen": true,
+         |          "maySetKeywords": true,
+         |          "mayCreateChild": true,
+         |          "mayRename": true,
+         |          "mayDelete": true,
+         |          "maySubmit": true
+         |        },
+         |        "isSubscribed": false
+         |      }],
+         |      "notFound": []
+         |    }, "c2"]
+         |  ]
+         |}""".stripMargin)
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 10/14: JAMES-3359 Mailbox/set update patch validation

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 22769e0aa47ad24a140b8b5b0c818fbb05dd4f2d
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 15:49:30 2020 +0700

    JAMES-3359 Mailbox/set update patch validation
---
 .../contract/MailboxSetMethodContract.scala        | 214 ++++++++++++++++++++-
 .../org/apache/james/jmap/mail/MailboxSet.scala    |  25 ++-
 .../james/jmap/method/MailboxSetMethod.scala       |   5 +-
 3 files changed, 237 insertions(+), 7 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 8d682ca..25b3859 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2262,6 +2262,218 @@ trait MailboxSetMethodContract {
   }
 
   @Test
+  def updateShouldFailWhenWrongJsonObject(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/name": ["newName"]
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidArguments",
+         |					"description": "Expecting a JSON string as an argument",
+         |					"properties": ["/name"]
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def updateShouldFailWhenUnknownProperty(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "/unknown": "newValue"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidArguments",
+         |					"description": "/unknown property do not exist thus cannot be updated",
+         |					"properties": ["/unknown"]
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def updateShouldFailWhenEmptyProperty(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "": "newValue"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    val message = "Invalid property specified in a patch object: Both predicates of (!isEmpty() && \\\"\\\".startsWith(\\\"/\\\")) failed. Left: Predicate isEmpty() did not fail. Right: Predicate failed: \\\"\\\".startsWith(\\\"/\\\")."
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidPatch",
+         |					"description": "$message"
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def updateShouldFailWhenInvalidProperty(server: GuiceJamesServer): Unit = {
+    val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId1.serialize()}": {
+        |                      "name": "newValue"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    val message = "Invalid property specified in a patch object: Right predicate of (!isEmpty(name) && \\\"name\\\".startsWith(\\\"/\\\")) failed: Predicate failed: \\\"name\\\".startsWith(\\\"/\\\")."
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId1.serialize()}": {
+         |					"type": "invalidPatch",
+         |					"description": "$message"
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
   def updateShouldFailWhenMailboxNameIsTooLong(server: GuiceJamesServer): Unit = {
     val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
     val request =
@@ -2523,6 +2735,4 @@ trait MailboxSetMethodContract {
          |	]
          |}""".stripMargin)
   }
-
-  // TODO invalid path handling (unknown property, invalid name)
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
index 3452209..e0964d6 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
@@ -22,8 +22,12 @@ package org.apache.james.jmap.mail
 import eu.timepit.refined
 import eu.timepit.refined.api.Refined
 import eu.timepit.refined.auto._
+import eu.timepit.refined.boolean.And
 import eu.timepit.refined.collection.NonEmpty
+import eu.timepit.refined.refineV
+import eu.timepit.refined.string.StartsWith
 import org.apache.james.jmap.mail.MailboxName.MailboxName
+import org.apache.james.jmap.mail.MailboxPatchObject.MailboxPatchObjectKey
 import org.apache.james.jmap.mail.MailboxSetRequest.{MailboxCreationId, UnparsedMailboxId}
 import org.apache.james.jmap.model.AccountId
 import org.apache.james.jmap.model.State.State
@@ -55,11 +59,20 @@ case class MailboxCreationRequest(name: MailboxName,
                                   isSubscribed: Option[IsSubscribed],
                                   rights: Option[Rights])
 
+object MailboxPatchObject {
+  type KeyConstraint = NonEmpty And StartsWith["/"]
+  type MailboxPatchObjectKey = String Refined KeyConstraint
+}
+
 case class MailboxPatchObject(value: Map[String, JsValue]) {
   def updates: Iterable[Either[PatchUpdateValidationException, Update]] = value.map({
     case (property, newValue) => property match {
       case "/name" => NameUpdate.parse(newValue)
-      case property => Left(UnsupportedPropertyUpdated(property))
+      case property =>
+        val refinedKey: Either[String, MailboxPatchObjectKey] = refineV(property)
+        refinedKey.fold[Either[PatchUpdateValidationException, Update]](
+          cause => Left(InvalidPropertyException(property = property, cause = s"Invalid property specified in a patch object: $cause")),
+          value => Left(UnsupportedPropertyUpdatedException(value)))
     }
   })
 }
@@ -77,6 +90,7 @@ case class MailboxSetResponse(accountId: AccountId,
 object MailboxSetError {
   val invalidArgumentValue: SetErrorType = "invalidArguments"
   val serverFailValue: SetErrorType = "serverFail"
+  val invalidPatchValue: SetErrorType = "invalidPatch"
   val notFoundValue: SetErrorType = "notFound"
   val mailboxHasEmailValue: SetErrorType = "mailboxHasEmail"
   val mailboxHasChildValue: SetErrorType = "mailboxHasChild"
@@ -87,6 +101,7 @@ object MailboxSetError {
   def notFound(description: Option[SetErrorDescription]) = MailboxSetError(notFoundValue, description, None)
   def mailboxHasEmail(description: Option[SetErrorDescription]) = MailboxSetError(mailboxHasEmailValue, description, None)
   def mailboxHasChild(description: Option[SetErrorDescription]) = MailboxSetError(mailboxHasChildValue, description, None)
+  def invalidPatch(description: Option[SetErrorDescription]) = MailboxSetError(invalidPatchValue, description, None)
   def forbidden(description: Option[SetErrorDescription], properties: Option[Properties]) = MailboxSetError(forbiddenValue, description, properties)
 }
 
@@ -114,7 +129,7 @@ case class MailboxUpdateResponse(value: JsObject)
 object NameUpdate {
   def parse(newValue: JsValue): Either[PatchUpdateValidationException, Update] = newValue match {
     case JsString(newName) => scala.Right(NameUpdate(newName))
-    case _ => Left(InvalidUpdate("name", "Expectint a JSON string as an argument"))
+    case _ => Left(InvalidUpdateException("/name", "Expecting a JSON string as an argument"))
   }
 }
 
@@ -122,5 +137,7 @@ sealed trait Update
 case class NameUpdate(newName: String) extends Update
 
 class PatchUpdateValidationException() extends IllegalArgumentException
-case class UnsupportedPropertyUpdated(property: String) extends PatchUpdateValidationException
-case class InvalidUpdate(property: String, cause: String) extends PatchUpdateValidationException
\ No newline at end of file
+case class UnsupportedPropertyUpdatedException(property: MailboxPatchObjectKey) extends PatchUpdateValidationException
+case class InvalidPropertyUpdatedException(property: MailboxPatchObjectKey) extends PatchUpdateValidationException
+case class InvalidPropertyException(property: String, cause: String) extends PatchUpdateValidationException
+case class InvalidUpdateException(property: MailboxPatchObjectKey, cause: String) extends PatchUpdateValidationException
\ No newline at end of file
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 6018c8f..2d10905 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -23,7 +23,7 @@ import eu.timepit.refined.auto._
 import javax.inject.Inject
 import org.apache.james.jmap.json.Serializer
 import org.apache.james.jmap.mail.MailboxSetRequest.{MailboxCreationId, UnparsedMailboxId}
-import org.apache.james.jmap.mail.{IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads}
+import org.apache.james.jmap.mail.{InvalidPropertyException, InvalidUpdateException, IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads, UnsupportedPropertyUpdatedException}
 import org.apache.james.jmap.model.CapabilityIdentifier.CapabilityIdentifier
 import org.apache.james.jmap.model.Invocation.{Arguments, MethodName}
 import org.apache.james.jmap.model.{ClientId, Id, Invocation, ServerId, State}
@@ -123,6 +123,9 @@ case class UpdateFailure(mailboxId: UnparsedMailboxId, exception: Throwable) ext
     case e: MailboxNotFoundException => MailboxSetError.notFound(Some(SetErrorDescription(e.getMessage)))
     case e: MailboxNameException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
     case e: MailboxExistsException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
+    case e: UnsupportedPropertyUpdatedException => MailboxSetError.invalidArgument(Some(SetErrorDescription(s"${e.property} property do not exist thus cannot be updated")), Some(Properties(List(e.property))))
+    case e: InvalidUpdateException => MailboxSetError.invalidArgument(Some(SetErrorDescription(s"${e.cause}")), Some(Properties(List(e.property))))
+    case e: InvalidPropertyException => MailboxSetError.invalidPatch(Some(SetErrorDescription(s"${e.cause}")))
     case e: SystemMailboxChangeException => MailboxSetError.invalidArgument(Some(SetErrorDescription("Invalid change to a system mailbox")), Some(Properties(List("/name"))))
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
   }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 14/14: JAMES-3357 Add sortOrder field in the created response for Mailbox/set method

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit ccfbe53a739bb1c0eca9397a6113258358f5371f
Author: Rene Cordier <rc...@linagora.com>
AuthorDate: Fri Aug 14 15:36:53 2020 +0700

    JAMES-3357 Add sortOrder field in the created response for Mailbox/set method
---
 .../james/jmap/rfc8621/contract/MailboxSetMethodContract.scala    | 8 ++++++++
 .../src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala    | 1 +
 .../scala/org/apache/james/jmap/method/MailboxSetMethod.scala     | 5 +++--
 3 files changed, 12 insertions(+), 2 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 582ce3a..a3bc8ca 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -450,6 +450,7 @@ trait MailboxSetMethodContract {
          |			"created": {
          |				"C42": {
          |					"id": "$mailboxId",
+         |          "sortOrder": 1000,
          |					"totalEmails": 0,
          |					"unreadEmails": 0,
          |					"totalThreads": 0,
@@ -673,6 +674,7 @@ trait MailboxSetMethodContract {
          |      "created": {
          |        "C42": {
          |          "id": "$mailboxId",
+         |          "sortOrder": 1000,
          |          "totalEmails": 0,
          |          "unreadEmails": 0,
          |          "totalThreads": 0,
@@ -819,6 +821,7 @@ trait MailboxSetMethodContract {
          |      "created": {
          |        "C42": {
          |          "id": "$mailboxId",
+         |          "sortOrder": 1000,
          |          "isSubscribed":true,
          |          "myRights":{"mayAddItems":true,"mayCreateChild":true,"mayDelete":true,"mayReadItems":true,"mayRemoveItems":true,"mayRename":true,"maySetKeywords":true,"maySetSeen":true,"maySubmit":true},
          |          "totalEmails":0,
@@ -886,6 +889,7 @@ trait MailboxSetMethodContract {
          |      "created": {
          |        "C42": {
          |          "id": "$mailboxId",
+         |          "sortOrder": 1000,
          |          "isSubscribed":true,
          |          "myRights":{"mayAddItems":true,"mayCreateChild":true,"mayDelete":true,"mayReadItems":true,"mayRemoveItems":true,"mayRename":true,"maySetKeywords":true,"maySetSeen":true,"maySubmit":true},
          |          "totalEmails":0,
@@ -1758,6 +1762,7 @@ trait MailboxSetMethodContract {
          |                "created": {
          |                    "C42": {
          |                        "id": "$parentId",
+         |                        "sortOrder": 1000,
          |                        "totalEmails": 0,
          |                        "unreadEmails": 0,
          |                        "totalThreads": 0,
@@ -1787,6 +1792,7 @@ trait MailboxSetMethodContract {
          |                "created": {
          |                    "C43": {
          |                        "id": "$childId",
+         |                        "sortOrder": 1000,
          |                        "totalEmails": 0,
          |                        "unreadEmails": 0,
          |                        "totalThreads": 0,
@@ -1876,6 +1882,7 @@ trait MailboxSetMethodContract {
          |      "created": {
          |        "C42": {
          |          "id": "$mailboxId",
+         |          "sortOrder": 1000,
          |          "totalEmails": 0,
          |          "unreadEmails": 0,
          |          "totalThreads": 0,
@@ -2137,6 +2144,7 @@ trait MailboxSetMethodContract {
          |      "created": {
          |        "C43": {
          |          "id": "$mailboxId",
+         |          "sortOrder": 1000,
          |          "totalEmails": 0,
          |          "unreadEmails": 0,
          |          "totalThreads": 0,
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
index e0964d6..b34af2c 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
@@ -109,6 +109,7 @@ case class MailboxSetError(`type`: SetErrorType, description: Option[SetErrorDes
 
 case class MailboxCreationResponse(id: MailboxId,
                                    role: Option[Role],//TODO see if we need to return this, if a role is set by the server during creation
+                                   sortOrder: SortOrder,
                                    totalEmails: TotalEmails,
                                    unreadEmails: UnreadEmails,
                                    totalThreads: TotalThreads,
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 2077181..d7bd881 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -23,7 +23,7 @@ import eu.timepit.refined.auto._
 import javax.inject.Inject
 import org.apache.james.jmap.json.Serializer
 import org.apache.james.jmap.mail.MailboxSetRequest.{MailboxCreationId, UnparsedMailboxId}
-import org.apache.james.jmap.mail.{InvalidPropertyException, InvalidUpdateException, IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, RemoveEmailsOnDestroy, SetErrorDescription, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads, UnsupportedPropertyUpdatedException}
+import org.apache.james.jmap.mail.{InvalidPropertyException, InvalidUpdateException, IsSubscribed, MailboxCreationRequest, MailboxCreationResponse, MailboxPatchObject, MailboxRights, MailboxSetError, MailboxSetRequest, MailboxSetResponse, MailboxUpdateResponse, NameUpdate, PatchUpdateValidationException, Properties, RemoveEmailsOnDestroy, SetErrorDescription, SortOrder, TotalEmails, TotalThreads, UnreadEmails, UnreadThreads, UnsupportedPropertyUpdatedException}
 import org.apache.james.jmap.model.CapabilityIdentifier.CapabilityIdentifier
 import org.apache.james.jmap.model.Invocation.{Arguments, MethodName}
 import org.apache.james.jmap.model.{ClientId, Id, Invocation, ServerId, State}
@@ -34,7 +34,7 @@ import org.apache.james.mailbox.model.{FetchGroup, MailboxId, MailboxPath, Messa
 import org.apache.james.mailbox.{MailboxManager, MailboxSession, MessageManager, Role, SubscriptionManager}
 import org.apache.james.metrics.api.MetricFactory
 import org.reactivestreams.Publisher
-import play.api.libs.json._
+import play.api.libs.json.{JsError, JsObject, JsPath, JsSuccess, Json, JsonValidationError}
 import reactor.core.scala.publisher.{SFlux, SMono}
 import reactor.core.scheduler.Schedulers
 
@@ -71,6 +71,7 @@ case class CreationResults(created: Seq[CreationResult]) {
   private def toCreationResponse(mailboxId: MailboxId): MailboxCreationResponse = MailboxCreationResponse(
     id = mailboxId,
     role = None,
+    sortOrder = SortOrder.defaultSortOrder,
     totalEmails = TotalEmails(0L),
     unreadEmails = UnreadEmails(0L),
     totalThreads = TotalThreads(0L),


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 11/14: JAMES-3359 Remove TABs from MailboxSetMethodContract

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit e882d7140af900ca542e04999daf598b776b8d3d
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 16:01:00 2020 +0700

    JAMES-3359 Remove TABs from MailboxSetMethodContract
---
 .../contract/MailboxSetMethodContract.scala        | 632 ++++++++++-----------
 1 file changed, 316 insertions(+), 316 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 25b3859..fdf59da 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -664,60 +664,60 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"created": {
-         |				"C42": {
-         |					"id": "$mailboxId",
-         |					"totalEmails": 0,
-         |					"unreadEmails": 0,
-         |					"totalThreads": 0,
-         |					"unreadThreads": 0,
-         |					"myRights": {
-         |						"mayReadItems": true,
-         |						"mayAddItems": true,
-         |						"mayRemoveItems": true,
-         |						"maySetSeen": true,
-         |						"maySetKeywords": true,
-         |						"mayCreateChild": true,
-         |						"mayRename": true,
-         |						"mayDelete": true,
-         |						"maySubmit": true
-         |					},
-         |					"isSubscribed": true
-         |				}
-         |			}
-         |		}, "c1"],
-         |		["Mailbox/get", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"state": "000001",
-         |			"list": [{
-         |				"id": "$mailboxId",
-         |				"name": "myMailbox",
-         |				"sortOrder": 1000,
-         |				"totalEmails": 0,
-         |				"unreadEmails": 0,
-         |				"totalThreads": 0,
-         |				"unreadThreads": 0,
-         |				"myRights": {
-         |					"mayReadItems": true,
-         |					"mayAddItems": true,
-         |					"mayRemoveItems": true,
-         |					"maySetSeen": true,
-         |					"maySetKeywords": true,
-         |					"mayCreateChild": true,
-         |					"mayRename": true,
-         |					"mayDelete": true,
-         |					"maySubmit": true
-         |				},
-         |				"isSubscribed": true
-         |			}],
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "created": {
+         |        "C42": {
+         |          "id": "$mailboxId",
+         |          "totalEmails": 0,
+         |          "unreadEmails": 0,
+         |          "totalThreads": 0,
+         |          "unreadThreads": 0,
+         |          "myRights": {
+         |            "mayReadItems": true,
+         |            "mayAddItems": true,
+         |            "mayRemoveItems": true,
+         |            "maySetSeen": true,
+         |            "maySetKeywords": true,
+         |            "mayCreateChild": true,
+         |            "mayRename": true,
+         |            "mayDelete": true,
+         |            "maySubmit": true
+         |          },
+         |          "isSubscribed": true
+         |        }
+         |      }
+         |    }, "c1"],
+         |    ["Mailbox/get", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "state": "000001",
+         |      "list": [{
+         |        "id": "$mailboxId",
+         |        "name": "myMailbox",
+         |        "sortOrder": 1000,
+         |        "totalEmails": 0,
+         |        "unreadEmails": 0,
+         |        "totalThreads": 0,
+         |        "unreadThreads": 0,
+         |        "myRights": {
+         |          "mayReadItems": true,
+         |          "mayAddItems": true,
+         |          "mayRemoveItems": true,
+         |          "maySetSeen": true,
+         |          "maySetKeywords": true,
+         |          "mayCreateChild": true,
+         |          "mayRename": true,
+         |          "mayDelete": true,
+         |          "maySubmit": true
+         |        },
+         |        "isSubscribed": true
+         |      }],
          |      "notFound":[]
-         |		}, "c2"]
-         |	]
+         |    }, "c2"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -1857,44 +1857,44 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notDestroyed": {
-         |				"#C42": {
-         |					"type": "invalidArguments",
-         |					"description": "#C42 is not a mailboxId: ClientId(#C42) was not used in previously defined creationIds"
-         |				}
-         |			}
-         |		}, "c2"],
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"created": {
-         |				"C42": {
-         |					"id": "$mailboxId",
-         |					"totalEmails": 0,
-         |					"unreadEmails": 0,
-         |					"totalThreads": 0,
-         |					"unreadThreads": 0,
-         |					"myRights": {
-         |						"mayReadItems": true,
-         |						"mayAddItems": true,
-         |						"mayRemoveItems": true,
-         |						"maySetSeen": true,
-         |						"maySetKeywords": true,
-         |						"mayCreateChild": true,
-         |						"mayRename": true,
-         |						"mayDelete": true,
-         |						"maySubmit": true
-         |					},
-         |					"isSubscribed": true
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notDestroyed": {
+         |        "#C42": {
+         |          "type": "invalidArguments",
+         |          "description": "#C42 is not a mailboxId: ClientId(#C42) was not used in previously defined creationIds"
+         |        }
+         |      }
+         |    }, "c2"],
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "created": {
+         |        "C42": {
+         |          "id": "$mailboxId",
+         |          "totalEmails": 0,
+         |          "unreadEmails": 0,
+         |          "totalThreads": 0,
+         |          "unreadThreads": 0,
+         |          "myRights": {
+         |            "mayReadItems": true,
+         |            "mayAddItems": true,
+         |            "mayRemoveItems": true,
+         |            "maySetSeen": true,
+         |            "maySetKeywords": true,
+         |            "mayCreateChild": true,
+         |            "mayRename": true,
+         |            "mayDelete": true,
+         |            "maySubmit": true
+         |          },
+         |          "isSubscribed": true
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -1930,19 +1930,19 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notDestroyed": {
-         |				"#C42": {
-         |					"type": "invalidArguments",
-         |					"description": "#C42 is not a mailboxId: ClientId(#C42) was not used in previously defined creationIds"
-         |				}
-         |			}
-         |		}, "c2"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notDestroyed": {
+         |        "#C42": {
+         |          "type": "invalidArguments",
+         |          "description": "#C42 is not a mailboxId: ClientId(#C42) was not used in previously defined creationIds"
+         |        }
+         |      }
+         |    }, "c2"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -1979,19 +1979,19 @@ trait MailboxSetMethodContract {
     val message = "# is not a mailboxId: Left predicate of ((!(0 < 1) && !(0 > 255)) && \\\"\\\".matches(\\\"^[#a-zA-Z0-9-_]*$\\\")) failed: Predicate taking size() = 0 failed: Left predicate of (!(0 < 1) && !(0 > 255)) failed: Predicate (0 < 1) did not fail."
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notDestroyed": {
-         |				"#": {
-         |					"type": "invalidArguments",
-         |					"description": "$message"
-         |				}
-         |			}
-         |		}, "c2"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notDestroyed": {
+         |        "#": {
+         |          "type": "invalidArguments",
+         |          "description": "$message"
+         |        }
+         |      }
+         |    }, "c2"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2039,42 +2039,42 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"updated": {
-         |				"1": {}
-         |			}
-         |		}, "c2"],
-         |		["Mailbox/get", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"state": "000001",
-         |			"list": [{
-         |				"id": "1",
-         |				"name": "newName",
-         |				"sortOrder": 1000,
-         |				"totalEmails": 0,
-         |				"unreadEmails": 0,
-         |				"totalThreads": 0,
-         |				"unreadThreads": 0,
-         |				"myRights": {
-         |					"mayReadItems": true,
-         |					"mayAddItems": true,
-         |					"mayRemoveItems": true,
-         |					"maySetSeen": true,
-         |					"maySetKeywords": true,
-         |					"mayCreateChild": true,
-         |					"mayRename": true,
-         |					"mayDelete": true,
-         |					"maySubmit": true
-         |				},
-         |				"isSubscribed": false
-         |			}],
-         |			"notFound": []
-         |		}, "c2"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "updated": {
+         |        "1": {}
+         |      }
+         |    }, "c2"],
+         |    ["Mailbox/get", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "state": "000001",
+         |      "list": [{
+         |        "id": "1",
+         |        "name": "newName",
+         |        "sortOrder": 1000,
+         |        "totalEmails": 0,
+         |        "unreadEmails": 0,
+         |        "totalThreads": 0,
+         |        "unreadThreads": 0,
+         |        "myRights": {
+         |          "mayReadItems": true,
+         |          "mayAddItems": true,
+         |          "mayRemoveItems": true,
+         |          "maySetSeen": true,
+         |          "maySetKeywords": true,
+         |          "mayCreateChild": true,
+         |          "mayRename": true,
+         |          "mayDelete": true,
+         |          "maySubmit": true
+         |        },
+         |        "isSubscribed": false
+         |      }],
+         |      "notFound": []
+         |    }, "c2"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2128,41 +2128,41 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"created": {
-         |				"C43": {
-         |					"id": "$mailboxId",
-         |					"totalEmails": 0,
-         |					"unreadEmails": 0,
-         |					"totalThreads": 0,
-         |					"unreadThreads": 0,
-         |					"myRights": {
-         |						"mayReadItems": true,
-         |						"mayAddItems": true,
-         |						"mayRemoveItems": true,
-         |						"maySetSeen": true,
-         |						"maySetKeywords": true,
-         |						"mayCreateChild": true,
-         |						"mayRename": true,
-         |						"mayDelete": true,
-         |						"maySubmit": true
-         |					},
-         |					"isSubscribed": true
-         |				}
-         |			}
-         |		}, "c1"],
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"updated": {
-         |				"$mailboxId": {}
-         |			}
-         |		}, "c2"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "created": {
+         |        "C43": {
+         |          "id": "$mailboxId",
+         |          "totalEmails": 0,
+         |          "unreadEmails": 0,
+         |          "totalThreads": 0,
+         |          "unreadThreads": 0,
+         |          "myRights": {
+         |            "mayReadItems": true,
+         |            "mayAddItems": true,
+         |            "mayRemoveItems": true,
+         |            "maySetSeen": true,
+         |            "maySetKeywords": true,
+         |            "mayCreateChild": true,
+         |            "mayRename": true,
+         |            "mayDelete": true,
+         |            "maySubmit": true
+         |          },
+         |          "isSubscribed": true
+         |        }
+         |      }
+         |    }, "c1"],
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "updated": {
+         |        "$mailboxId": {}
+         |      }
+         |    }, "c2"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2244,20 +2244,20 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidArguments",
-         |					"description": "Mailbox with name=#private:bob@domain.tld:newName already exists.",
-         |					"properties": ["/name"]
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "Mailbox with name=#private:bob@domain.tld:newName already exists.",
+         |          "properties": ["/name"]
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2297,20 +2297,20 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidArguments",
-         |					"description": "Expecting a JSON string as an argument",
-         |					"properties": ["/name"]
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "Expecting a JSON string as an argument",
+         |          "properties": ["/name"]
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2350,20 +2350,20 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidArguments",
-         |					"description": "/unknown property do not exist thus cannot be updated",
-         |					"properties": ["/unknown"]
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "/unknown property do not exist thus cannot be updated",
+         |          "properties": ["/unknown"]
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2404,19 +2404,19 @@ trait MailboxSetMethodContract {
     val message = "Invalid property specified in a patch object: Both predicates of (!isEmpty() && \\\"\\\".startsWith(\\\"/\\\")) failed. Left: Predicate isEmpty() did not fail. Right: Predicate failed: \\\"\\\".startsWith(\\\"/\\\")."
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidPatch",
-         |					"description": "$message"
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidPatch",
+         |          "description": "$message"
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2457,19 +2457,19 @@ trait MailboxSetMethodContract {
     val message = "Invalid property specified in a patch object: Right predicate of (!isEmpty(name) && \\\"name\\\".startsWith(\\\"/\\\")) failed: Predicate failed: \\\"name\\\".startsWith(\\\"/\\\")."
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidPatch",
-         |					"description": "$message"
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidPatch",
+         |          "description": "$message"
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2509,20 +2509,20 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "invalidArguments",
-         |					"description": "Mailbox name exceeds maximum size of 200 characters",
-         |					"properties": ["/name"]
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "Mailbox name exceeds maximum size of 200 characters",
+         |          "properties": ["/name"]
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2562,19 +2562,19 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "notFound",
-         |					"description": "${mailboxId1.serialize()} can not be found"
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "notFound",
+         |          "description": "${mailboxId1.serialize()} can not be found"
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2617,19 +2617,19 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId1.serialize()}": {
-         |					"type": "notFound",
-         |					"description": "#private:andre@domain.tld:previousName"
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId1.serialize()}": {
+         |          "type": "notFound",
+         |          "description": "#private:andre@domain.tld:previousName"
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2670,20 +2670,20 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
-         |			"notUpdated": {
-         |				"${mailboxId.serialize()}": {
-         |					"type": "invalidArguments",
-         |					"description": "Invalid change to a system mailbox",
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
+         |      "notUpdated": {
+         |        "${mailboxId.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "Invalid change to a system mailbox",
          |          "properties":["/name"]
-         |				}
-         |			}
-         |		}, "c1"]
-         |	]
+         |        }
+         |      }
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 
@@ -2720,19 +2720,19 @@ trait MailboxSetMethodContract {
 
     assertThatJson(response).isEqualTo(
       s"""{
-         |	"sessionState": "75128aab4b1b",
-         |	"methodResponses": [
-         |		["Mailbox/set", {
-         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
-         |			"newState": "000001",
+         |  "sessionState": "75128aab4b1b",
+         |  "methodResponses": [
+         |    ["Mailbox/set", {
+         |      "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |      "newState": "000001",
          |      "notDestroyed": {
          |        "${mailboxId.serialize()}": {
          |          "type": "invalidArguments",
          |          "description": "System mailboxes cannot be destroyed"
          |        }
          |      }
-         |		}, "c1"]
-         |	]
+         |    }, "c1"]
+         |  ]
          |}""".stripMargin)
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 04/14: JAMES-3359 Mailbox/set update should accept creationIds

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit 2b1d2e71090b9fc65ff72ad6c1c02bd711e412a7
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 14:02:37 2020 +0700

    JAMES-3359 Mailbox/set update should accept creationIds
---
 .../contract/MailboxSetMethodContract.scala        | 89 +++++++++++++++++++++-
 .../org/apache/james/jmap/json/Serializer.scala    | 21 ++++-
 .../org/apache/james/jmap/mail/MailboxSet.scala    |  4 +-
 .../james/jmap/method/MailboxSetMethod.scala       | 13 ++--
 4 files changed, 116 insertions(+), 11 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index b1853b9..180a198 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2077,7 +2077,94 @@ trait MailboxSetMethodContract {
          |}""".stripMargin)
   }
 
-  // TODO update using creation ids
+  @Test
+  def updateShouldAcceptCreationIds(server: GuiceJamesServer): Unit = {
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "create": {
+        |                    "C43": {
+        |                      "name": "mailbox"
+        |                    }
+        |                }
+        |           },
+        |    "c1"],
+        |      ["Mailbox/set",
+        |          {
+        |               "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |               "update": {
+        |                 "#C43" : {
+        |                   "/name": "newName"
+        |                 }
+        |               }
+        |          },
+        |     "c2"]
+        |   ]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    val mailboxId: String = server.getProbe(classOf[MailboxProbeImpl])
+      .getMailboxId("#private", BOB.asString(), "newName")
+      .serialize()
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"created": {
+         |				"C43": {
+         |					"id": "$mailboxId",
+         |					"totalEmails": 0,
+         |					"unreadEmails": 0,
+         |					"totalThreads": 0,
+         |					"unreadThreads": 0,
+         |					"myRights": {
+         |						"mayReadItems": true,
+         |						"mayAddItems": true,
+         |						"mayRemoveItems": true,
+         |						"maySetSeen": true,
+         |						"maySetKeywords": true,
+         |						"mayCreateChild": true,
+         |						"mayRename": true,
+         |						"mayDelete": true,
+         |						"maySubmit": true
+         |					},
+         |					"isSubscribed": true
+         |				}
+         |			}
+         |		}, "c1"],
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"updated": {
+         |				"$mailboxId": {}
+         |			}
+         |		}, "c2"]
+         |	]
+         |}""".stripMargin)
+  }
+
   // TODO invalid path handling (unknown property, invalid name)
   // TODO mailbox already exists, too long name
   // TODO disable destroy / rename of sustem mailbox
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
index b655527..a813bf3 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/json/Serializer.scala
@@ -241,9 +241,24 @@ class Serializer @Inject() (mailboxIdFactory: MailboxId.Factory) {
   implicit val mailboxCreationRequest: Reads[MailboxCreationRequest] = Json.reads[MailboxCreationRequest]
   private implicit val mailboxPatchObject: Reads[MailboxPatchObject] = Json.valueReads[MailboxPatchObject]
 
-  private implicit val mapPatchObjectByMailboxIdReads: Reads[Map[MailboxId, MailboxPatchObject]] = _.validate[Map[String, MailboxPatchObject]]
-    .map(mapWithStringKey => mapWithStringKey
-      .map(keyValue => (mailboxIdFactory.fromString(keyValue._1), keyValue._2)))
+  private implicit val mapPatchObjectByMailboxIdReads: Reads[Map[UnparsedMailboxId, MailboxPatchObject]] = _.validate[Map[String, MailboxPatchObject]]
+    .flatMap(mapWithStringKey =>{
+      mapWithStringKey
+        .foldLeft[Either[JsError, Map[UnparsedMailboxId, MailboxPatchObject]]](scala.util.Right[JsError, Map[UnparsedMailboxId, MailboxPatchObject]](Map.empty))((acc: Either[JsError, Map[UnparsedMailboxId, MailboxPatchObject]], keyValue) => {
+          acc match {
+            case error@Left(_) => error
+            case scala.util.Right(validatedAcc) =>
+              val refinedKey: Either[String, UnparsedMailboxId] = refineV(keyValue._1)
+              refinedKey match {
+                case Left(error) => Left(JsError(error))
+                case scala.util.Right(unparsedMailboxId) => scala.util.Right(validatedAcc + (unparsedMailboxId -> keyValue._2))
+              }
+          }
+        }) match {
+        case Left(jsError) => jsError
+        case scala.util.Right(value) => JsSuccess(value)
+      }
+    })
 
   private implicit val mapCreationRequestByMailBoxCreationId: Reads[Map[MailboxCreationId, JsObject]] = _.validate[Map[String, JsObject]]
     .flatMap(mapWithStringKey => {
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
index 51821dd..3452209 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/mail/MailboxSet.scala
@@ -34,7 +34,7 @@ import play.api.libs.json.{JsObject, JsString, JsValue}
 case class MailboxSetRequest(accountId: AccountId,
                              ifInState: Option[State],
                              create: Option[Map[MailboxCreationId, JsObject]],
-                             update: Option[Map[MailboxId, MailboxPatchObject]],
+                             update: Option[Map[UnparsedMailboxId, MailboxPatchObject]],
                              destroy: Option[Seq[UnparsedMailboxId]],
                              onDestroyRemoveEmails: Option[RemoveEmailsOnDestroy])
 
@@ -71,7 +71,7 @@ case class MailboxSetResponse(accountId: AccountId,
                               updated: Option[Map[MailboxId, MailboxUpdateResponse]],
                               destroyed: Option[Seq[MailboxId]],
                               notCreated: Option[Map[MailboxCreationId, MailboxSetError]],
-                              notUpdated: Option[Map[MailboxId, MailboxSetError]],
+                              notUpdated: Option[Map[UnparsedMailboxId, MailboxSetError]],
                               notDestroyed: Option[Map[UnparsedMailboxId, MailboxSetError]])
 
 object MailboxSetError {
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index 81fc67d..e774a0b 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -116,7 +116,7 @@ case class DeletionResults(results: Seq[DeletionResult]) {
 
 sealed trait UpdateResult
 case class UpdateSuccess(mailboxId: MailboxId) extends UpdateResult
-case class UpdateFailure(mailboxId: MailboxId, exception: Throwable) extends UpdateResult {
+case class UpdateFailure(mailboxId: UnparsedMailboxId, exception: Throwable) extends UpdateResult {
   def asMailboxSetError: MailboxSetError = exception match {
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
   }
@@ -127,7 +127,7 @@ case class UpdateResults(results: Seq[UpdateResult]) {
       case success: UpdateSuccess => Some((success.mailboxId, MailboxSetResponse.empty))
       case _ => None
     }).toMap
-  def notUpdated: Map[MailboxId, MailboxSetError] = results.flatMap(result => result match {
+  def notUpdated: Map[UnparsedMailboxId, MailboxSetError] = results.flatMap(result => result match {
     case failure: UpdateFailure => Some(failure.mailboxId, failure.asMailboxSetError)
     case _ => None
   }).toMap
@@ -157,8 +157,11 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
                               processingContext: ProcessingContext): SMono[UpdateResults] = {
     SFlux.fromIterable(mailboxSetRequest.update.getOrElse(Seq()))
       .flatMap({
-        case (mailboxId: MailboxId, patch: MailboxPatchObject) => updateMailbox(mailboxSession, mailboxId, patch)
-          .onErrorResume(e => SMono.just(UpdateFailure(mailboxId, e)))
+        case (unparsedMailboxId: UnparsedMailboxId, patch: MailboxPatchObject) =>
+          processingContext.resolveMailboxId(unparsedMailboxId, mailboxIdFactory).fold(
+              e => SMono.just(UpdateFailure(unparsedMailboxId, e)),
+              mailboxId => updateMailbox(mailboxSession, mailboxId, patch))
+            .onErrorResume(e => SMono.just(UpdateFailure(unparsedMailboxId, e)))
       })
       .collectSeq()
       .map(UpdateResults)
@@ -181,7 +184,7 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
 
     def renameMailbox: SMono[UpdateResult] = updateMailboxPath(maiboxId, maybeNameUpdate, mailboxSession)
 
-    maybeParseException.map(e => SMono.just[UpdateResult](UpdateFailure(maiboxId, e)))
+    maybeParseException.map(e => SMono.raiseError[UpdateResult](e))
       .getOrElse(renameMailbox)
   }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 06/14: JAMES-3359 Mailbox/set update should handle subscription

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit fa160b841fbe7e8095398903928cf349dadfb59e
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 14:44:20 2020 +0700

    JAMES-3359 Mailbox/set update should handle subscription
    
    The test requires mailbox/set create subscription support to pass and is thus disabled for now.
---
 .../contract/MailboxSetMethodContract.scala        | 43 +++++++++++++++++++++-
 1 file changed, 42 insertions(+), 1 deletion(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index fefaebc..d57b247 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -38,6 +38,7 @@ import org.apache.james.mime4j.dom.Message
 import org.apache.james.modules.{ACLProbeImpl, MailboxProbeImpl}
 import org.apache.james.utils.DataProbeImpl
 import org.assertj.core.api.Assertions
+import org.assertj.core.api.Assertions.assertThat
 import org.hamcrest.Matchers.{equalTo, hasSize}
 import org.junit.jupiter.api.{BeforeEach, Disabled, Test}
 
@@ -2166,6 +2167,47 @@ trait MailboxSetMethodContract {
   }
 
   @Test
+  def updateShouldRenameSubscriptions(server: GuiceJamesServer): Unit = {
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "create": {
+        |                    "C43": {
+        |                      "name": "mailbox"
+        |                    }
+        |                }
+        |           },
+        |    "c1"],
+        |      ["Mailbox/set",
+        |          {
+        |               "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |               "update": {
+        |                 "#C43" : {
+        |                   "/name": "newName"
+        |                 }
+        |               }
+        |          },
+        |     "c2"]
+        |   ]
+        |}
+        |""".stripMargin
+
+    `with`()
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+      .post
+
+    assertThat(server.getProbe(classOf[MailboxProbeImpl]).listSubscriptions(BOB.asString()))
+      .contains("newName")
+      .doesNotContain("mailbox")
+  }
+
+  @Test
   def updateShouldFailWhenTargetMailboxAlreadyExist(server: GuiceJamesServer): Unit = {
     val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "previousName"))
     server.getProbe(classOf[MailboxProbeImpl]).createMailbox(MailboxPath.forUser(BOB, "newName"))
@@ -2278,6 +2320,5 @@ trait MailboxSetMethodContract {
 
   // TODO invalid path handling (unknown property, invalid name)
   // TODO disable destroy / rename of system mailbox
-  // TODO test that renames keeps subscriptions
   // TODO renaming delegated mailboxes is not allowed
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 02/14: JAMES-3361 ImapHostSystem should use owner's session to set rights for mailbox

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit f5d397ca40518df5b6b39443f60a34653bc4b423
Author: LanKhuat <dl...@linagora.com>
AuthorDate: Thu Aug 20 10:36:39 2020 +0700

    JAMES-3361 ImapHostSystem should use owner's session to set rights for mailbox
---
 .../src/main/java/org/apache/james/mpt/host/JamesImapHostSystem.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/mpt/impl/imap-mailbox/core/src/main/java/org/apache/james/mpt/host/JamesImapHostSystem.java b/mpt/impl/imap-mailbox/core/src/main/java/org/apache/james/mpt/host/JamesImapHostSystem.java
index b15d28e..cd1e0d7 100644
--- a/mpt/impl/imap-mailbox/core/src/main/java/org/apache/james/mpt/host/JamesImapHostSystem.java
+++ b/mpt/impl/imap-mailbox/core/src/main/java/org/apache/james/mpt/host/JamesImapHostSystem.java
@@ -118,7 +118,7 @@ public abstract class JamesImapHostSystem implements ImapHostSystem, GrantRights
                 .forUser(username)
                 .rights(rights)
                 .asAddition()),
-            mailboxManager.createSystemSession(username));
+            mailboxSession);
         mailboxManager.logout(mailboxSession);
         mailboxManager.endProcessingRequest(mailboxSession);
     }


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org


[james-project] 08/14: JAMES-3359 Mailbox/set update+destroy should fail on system mailboxes

Posted by rc...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

rcordier pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/james-project.git

commit b11b3f260f85f76bd850a53876482ecd2dd84ec2
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Aug 17 15:14:40 2020 +0700

    JAMES-3359 Mailbox/set update+destroy should fail on system mailboxes
---
 .../contract/MailboxSetMethodContract.scala        | 106 ++++++++++++++++++++-
 .../james/jmap/method/MailboxSetMethod.scala       |  23 +++--
 2 files changed, 121 insertions(+), 8 deletions(-)

diff --git a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
index 3436f8f..cebae49 100644
--- a/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
+++ b/server/protocols/jmap-rfc-8621-integration-tests/jmap-rfc-8621-integration-tests-common/src/main/scala/org/apache/james/jmap/rfc8621/contract/MailboxSetMethodContract.scala
@@ -2371,7 +2371,7 @@ trait MailboxSetMethodContract {
   }
 
   @Test
-  def updateShouldNotRenameDelegtedMailboxes(server: GuiceJamesServer): Unit = {
+  def updateShouldNotRenameDelegatedMailboxes(server: GuiceJamesServer): Unit = {
     val path = MailboxPath.forUser(ANDRE, "previousName")
     val mailboxId1: MailboxId = server.getProbe(classOf[MailboxProbeImpl]).createMailbox(path)
     server.getProbe(classOf[ACLProbeImpl])
@@ -2425,6 +2425,108 @@ trait MailboxSetMethodContract {
          |}""".stripMargin)
   }
 
+  @Test
+  def updateShouldNotRenameSystemMailboxes(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "INBOX"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "update": {
+        |                    "${mailboxId.serialize()}": {
+        |                      "/name": "newName"
+        |                    }
+        |                }
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |			"notUpdated": {
+         |				"${mailboxId.serialize()}": {
+         |					"type": "invalidArguments",
+         |					"description": "Invalid change to a system mailbox",
+         |          "properties":{"value":["/name"]}
+         |				}
+         |			}
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
+  @Test
+  def destroyShouldNotRemoveSystemMailboxes(server: GuiceJamesServer): Unit = {
+    val mailboxId: MailboxId = server.getProbe(classOf[MailboxProbeImpl])
+      .createMailbox(MailboxPath.forUser(BOB, "INBOX"))
+    val request =
+      s"""
+        |{
+        |   "using": [ "urn:ietf:params:jmap:core", "urn:ietf:params:jmap:mail" ],
+        |   "methodCalls": [
+        |       ["Mailbox/set",
+        |           {
+        |                "accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+        |                "destroy": ["${mailboxId.serialize()}"]
+        |           },
+        |    "c1"]]
+        |}
+        |""".stripMargin
+
+    val response = `given`
+      .header(ACCEPT.toString, ACCEPT_RFC8621_VERSION_HEADER)
+      .body(request)
+    .when
+      .post
+    .`then`
+      .log().ifValidationFails()
+      .statusCode(SC_OK)
+      .contentType(JSON)
+      .extract
+      .body
+      .asString
+
+    assertThatJson(response).isEqualTo(
+      s"""{
+         |	"sessionState": "75128aab4b1b",
+         |	"methodResponses": [
+         |		["Mailbox/set", {
+         |			"accountId": "29883977c13473ae7cb7678ef767cbfbaffc8a44a6e463d971d23a65c1dc4af6",
+         |			"newState": "000001",
+         |      "notDestroyed": {
+         |        "${mailboxId.serialize()}": {
+         |          "type": "invalidArguments",
+         |          "description": "System mailboxes cannot be destroyed"
+         |        }
+         |      }
+         |		}, "c1"]
+         |	]
+         |}""".stripMargin)
+  }
+
   // TODO invalid path handling (unknown property, invalid name)
-  // TODO disable destroy / rename of system mailbox
 }
diff --git a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
index ea79a9c..6018c8f 100644
--- a/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
+++ b/server/protocols/jmap-rfc-8621/src/main/scala/org/apache/james/jmap/method/MailboxSetMethod.scala
@@ -31,7 +31,7 @@ import org.apache.james.jmap.routes.ProcessingContext
 import org.apache.james.mailbox.MailboxManager.RenameOption
 import org.apache.james.mailbox.exception.{InsufficientRightsException, MailboxExistsException, MailboxNameException, MailboxNotFoundException}
 import org.apache.james.mailbox.model.{FetchGroup, MailboxId, MailboxPath, MessageRange}
-import org.apache.james.mailbox.{MailboxManager, MailboxSession, MessageManager, SubscriptionManager}
+import org.apache.james.mailbox.{MailboxManager, MailboxSession, MessageManager, Role, SubscriptionManager}
 import org.apache.james.metrics.api.MetricFactory
 import org.reactivestreams.Publisher
 import play.api.libs.json._
@@ -41,6 +41,7 @@ import reactor.core.scheduler.Schedulers
 import scala.jdk.CollectionConverters._
 
 case class MailboxHasMailException(mailboxId: MailboxId) extends Exception
+case class SystemMailboxChangeException(mailboxId: MailboxId) extends Exception
 case class MailboxHasChildException(mailboxId: MailboxId) extends Exception
 case class MailboxCreationParseException(mailboxSetError: MailboxSetError) extends Exception
 
@@ -95,6 +96,7 @@ case class DeletionFailure(mailboxId: UnparsedMailboxId, exception: Throwable) e
     case e: MailboxNotFoundException => MailboxSetError.notFound(Some(SetErrorDescription(e.getMessage)))
     case e: MailboxHasMailException => MailboxSetError.mailboxHasEmail(Some(SetErrorDescription(s"${e.mailboxId.serialize} is not empty")))
     case e: MailboxHasChildException => MailboxSetError.mailboxHasChild(Some(SetErrorDescription(s"${e.mailboxId.serialize} has child mailboxes")))
+    case e: SystemMailboxChangeException => MailboxSetError.invalidArgument(Some(SetErrorDescription("System mailboxes cannot be destroyed")), None)
     case e: IllegalArgumentException => MailboxSetError.invalidArgument(Some(SetErrorDescription(s"${mailboxId} is not a mailboxId: ${e.getMessage}")), None)
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
   }
@@ -121,6 +123,7 @@ case class UpdateFailure(mailboxId: UnparsedMailboxId, exception: Throwable) ext
     case e: MailboxNotFoundException => MailboxSetError.notFound(Some(SetErrorDescription(e.getMessage)))
     case e: MailboxNameException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
     case e: MailboxExistsException => MailboxSetError.invalidArgument(Some(SetErrorDescription(e.getMessage)), Some(Properties(List("/name"))))
+    case e: SystemMailboxChangeException => MailboxSetError.invalidArgument(Some(SetErrorDescription("Invalid change to a system mailbox")), Some(Properties(List("/name"))))
     case _ => MailboxSetError.serverFail(Some(SetErrorDescription(exception.getMessage)), None)
   }
 }
@@ -191,19 +194,22 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
       .getOrElse(renameMailbox)
   }
 
-  private def updateMailboxPath(maiboxId: MailboxId, maybeNameUpdate: Option[NameUpdate], mailboxSession: MailboxSession): SMono[UpdateResult] = {
+  private def updateMailboxPath(mailboxId: MailboxId, maybeNameUpdate: Option[NameUpdate], mailboxSession: MailboxSession): SMono[UpdateResult] = {
     maybeNameUpdate.map(nameUpdate => {
       SMono.fromCallable(() => {
-        val mailbox = mailboxManager.getMailbox(maiboxId, mailboxSession)
-        mailboxManager.renameMailbox(maiboxId,
+        val mailbox = mailboxManager.getMailbox(mailboxId, mailboxSession)
+        if (isASystemMailbox(mailbox)) {
+          throw SystemMailboxChangeException(mailboxId)
+        }
+        mailboxManager.renameMailbox(mailboxId,
           computeMailboxPath(mailbox, nameUpdate, mailboxSession),
           RenameOption.RENAME_SUBSCRIPTIONS,
           mailboxSession)
-      }).`then`(SMono.just[UpdateResult](UpdateSuccess(maiboxId)))
+      }).`then`(SMono.just[UpdateResult](UpdateSuccess(mailboxId)))
         .subscribeOn(Schedulers.elastic())
     })
       // No updated properties passed. Noop.
-      .getOrElse(SMono.just[UpdateResult](UpdateSuccess(maiboxId)))
+      .getOrElse(SMono.just[UpdateResult](UpdateSuccess(mailboxId)))
   }
 
   private def computeMailboxPath(mailbox: MessageManager, nameUpdate: NameUpdate, mailboxSession: MailboxSession): MailboxPath = {
@@ -236,6 +242,9 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
 
   private def delete(mailboxSession: MailboxSession, id: MailboxId): Unit = {
     val mailbox = mailboxManager.getMailbox(id, mailboxSession)
+    if (isASystemMailbox(mailbox)) {
+      throw SystemMailboxChangeException(id)
+    }
     if (mailbox.getMessages(MessageRange.all(), FetchGroup.MINIMAL, mailboxSession).hasNext) {
       throw MailboxHasMailException(id)
     }
@@ -246,6 +255,8 @@ class MailboxSetMethod @Inject()(serializer: Serializer,
     subscriptionManager.unsubscribe(mailboxSession, deletedMailbox.getName)
   }
 
+  private def isASystemMailbox(mailbox: MessageManager): Boolean = Role.from(mailbox.getMailboxPath.getName).isPresent
+
   private def createMailboxes(mailboxSession: MailboxSession,
                               mailboxSetRequest: MailboxSetRequest,
                               processingContext: ProcessingContext): SMono[CreationResults] = {


---------------------------------------------------------------------
To unsubscribe, e-mail: server-dev-unsubscribe@james.apache.org
For additional commands, e-mail: server-dev-help@james.apache.org