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 bt...@apache.org on 2019/12/18 11:29:49 UTC

[james-project] 02/06: JAMES-2993 ReIndexing routes sould be resource oriented

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

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

commit be34263dd35267c9cd858a5c31cd4bc446c5e575
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Mon Dec 16 13:27:38 2019 +0700

    JAMES-2993 ReIndexing routes sould be resource oriented
    
     - Rename ReIndexingRoutes into MailboxesRoutes
     - Rename MessageIdReindexingRoutes into message routes
     - Move user mailbox reIndexing
    
    Was `XPOST /mailboxes?user=bob@apache.org&task=reIndex`
    
    Now `XPOST /users/bob@apache.org/mailboxes?task=reIndex`
---
 .../org/apache/james/CassandraJamesServerMain.java |   8 +-
 .../java/org/apache/james/JPAJamesServerMain.java  |   4 +-
 ...exingModule.java => MailboxesRoutesModule.java} |   6 +-
 ...dexingModule.java => MessagesRoutesModule.java} |   6 +-
 ...dminServerTaskSerializationIntegrationTest.java |   3 +-
 ...{ReindexingRoutes.java => MailboxesRoutes.java} |  44 +--
 ...IdReindexingRoutes.java => MessagesRoutes.java} |   8 +-
 .../james/webadmin/routes/UserMailboxesRoutes.java |  45 ++-
 ...ingRoutesTest.java => MailboxesRoutesTest.java} | 356 +----------------
 .../webadmin/routes/MalformedUrlRoutesTest.java    |   8 +-
 .../james/webadmin/routes/MessageRoutesTest.java   | 235 +++++++++++
 .../webadmin/routes/UserMailboxesRoutesTest.java   | 243 +++++++++++-
 src/site/markdown/server/manage-webadmin.md        | 431 +++++++++++----------
 13 files changed, 759 insertions(+), 638 deletions(-)

diff --git a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/CassandraJamesServerMain.java b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/CassandraJamesServerMain.java
index d6c1a37..84953d0 100644
--- a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/CassandraJamesServerMain.java
+++ b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/CassandraJamesServerMain.java
@@ -62,8 +62,8 @@ import org.apache.james.modules.server.JMXServerModule;
 import org.apache.james.modules.server.MailQueueRoutesModule;
 import org.apache.james.modules.server.MailRepositoriesRoutesModule;
 import org.apache.james.modules.server.MailboxRoutesModule;
-import org.apache.james.modules.server.MessageIdReIndexingModule;
-import org.apache.james.modules.server.ReIndexingModule;
+import org.apache.james.modules.server.MailboxesRoutesModule;
+import org.apache.james.modules.server.MessagesRoutesModule;
 import org.apache.james.modules.server.SieveRoutesModule;
 import org.apache.james.modules.server.SwaggerRoutesModule;
 import org.apache.james.modules.server.TaskManagerModule;
@@ -92,8 +92,8 @@ public class CassandraJamesServerMain {
         new WebAdminServerModule(),
         new DLPRoutesModule(),
         new SieveRoutesModule(),
-        new ReIndexingModule(),
-        new MessageIdReIndexingModule());
+        new MailboxesRoutesModule(),
+        new MessagesRoutesModule());
 
     public static final Module PROTOCOLS = Modules.combine(
         new CassandraJmapModule(),
diff --git a/server/container/guice/jpa-guice/src/main/java/org/apache/james/JPAJamesServerMain.java b/server/container/guice/jpa-guice/src/main/java/org/apache/james/JPAJamesServerMain.java
index 4347b9a..2fb60b8 100644
--- a/server/container/guice/jpa-guice/src/main/java/org/apache/james/JPAJamesServerMain.java
+++ b/server/container/guice/jpa-guice/src/main/java/org/apache/james/JPAJamesServerMain.java
@@ -41,9 +41,9 @@ import org.apache.james.modules.server.JMXServerModule;
 import org.apache.james.modules.server.MailQueueRoutesModule;
 import org.apache.james.modules.server.MailRepositoriesRoutesModule;
 import org.apache.james.modules.server.MailboxRoutesModule;
+import org.apache.james.modules.server.MailboxesRoutesModule;
 import org.apache.james.modules.server.NoJwtModule;
 import org.apache.james.modules.server.RawPostDequeueDecoratorModule;
-import org.apache.james.modules.server.ReIndexingModule;
 import org.apache.james.modules.server.SieveRoutesModule;
 import org.apache.james.modules.server.SwaggerRoutesModule;
 import org.apache.james.modules.server.TaskManagerModule;
@@ -64,7 +64,7 @@ public class JPAJamesServerMain {
         new MailRepositoriesRoutesModule(),
         new SwaggerRoutesModule(),
         new SieveRoutesModule(),
-        new ReIndexingModule());
+        new MailboxesRoutesModule());
 
     public static final Module PROTOCOLS = Modules.combine(
         new IMAPServerModule(),
diff --git a/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessageIdReIndexingModule.java b/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MailboxesRoutesModule.java
similarity index 88%
rename from server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessageIdReIndexingModule.java
rename to server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MailboxesRoutesModule.java
index 19f09ba..9075f34 100644
--- a/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessageIdReIndexingModule.java
+++ b/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MailboxesRoutesModule.java
@@ -20,15 +20,15 @@
 package org.apache.james.modules.server;
 
 import org.apache.james.webadmin.Routes;
-import org.apache.james.webadmin.routes.MessageIdReindexingRoutes;
+import org.apache.james.webadmin.routes.MailboxesRoutes;
 
 import com.google.inject.AbstractModule;
 import com.google.inject.multibindings.Multibinder;
 
-public class MessageIdReIndexingModule extends AbstractModule {
+public class MailboxesRoutesModule extends AbstractModule {
     @Override
     protected void configure() {
         Multibinder<Routes> routesMultibinder = Multibinder.newSetBinder(binder(), Routes.class);
-        routesMultibinder.addBinding().to(MessageIdReindexingRoutes.class);
+        routesMultibinder.addBinding().to(MailboxesRoutes.class);
     }
 }
diff --git a/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/ReIndexingModule.java b/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessagesRoutesModule.java
similarity index 89%
rename from server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/ReIndexingModule.java
rename to server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessagesRoutesModule.java
index d1902ad..8af3a00 100644
--- a/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/ReIndexingModule.java
+++ b/server/container/guice/protocols/webadmin-mailbox/src/main/java/org/apache/james/modules/server/MessagesRoutesModule.java
@@ -20,15 +20,15 @@
 package org.apache.james.modules.server;
 
 import org.apache.james.webadmin.Routes;
-import org.apache.james.webadmin.routes.ReindexingRoutes;
+import org.apache.james.webadmin.routes.MessagesRoutes;
 
 import com.google.inject.AbstractModule;
 import com.google.inject.multibindings.Multibinder;
 
-public class ReIndexingModule extends AbstractModule {
+public class MessagesRoutesModule extends AbstractModule {
     @Override
     protected void configure() {
         Multibinder<Routes> routesMultibinder = Multibinder.newSetBinder(binder(), Routes.class);
-        routesMultibinder.addBinding().to(ReindexingRoutes.class);
+        routesMultibinder.addBinding().to(MessagesRoutes.class);
     }
 }
diff --git a/server/protocols/webadmin-integration-test/src/test/java/org/apache/james/webadmin/integration/WebAdminServerTaskSerializationIntegrationTest.java b/server/protocols/webadmin-integration-test/src/test/java/org/apache/james/webadmin/integration/WebAdminServerTaskSerializationIntegrationTest.java
index 9d633b0..1b1f007 100644
--- a/server/protocols/webadmin-integration-test/src/test/java/org/apache/james/webadmin/integration/WebAdminServerTaskSerializationIntegrationTest.java
+++ b/server/protocols/webadmin-integration-test/src/test/java/org/apache/james/webadmin/integration/WebAdminServerTaskSerializationIntegrationTest.java
@@ -311,9 +311,8 @@ public class WebAdminServerTaskSerializationIntegrationTest {
     @Test
     public void userReindexingShouldComplete() {
         String taskId = with()
-                .queryParam("user", USERNAME)
                 .queryParam("task", "reIndex")
-            .post("/mailboxes")
+            .post("users/" + USERNAME + "/mailboxes")
                 .jsonPath()
                 .get("taskId");
 
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/ReindexingRoutes.java b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MailboxesRoutes.java
similarity index 86%
rename from server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/ReindexingRoutes.java
rename to server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MailboxesRoutes.java
index f4223cb..5d644fd 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/ReindexingRoutes.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MailboxesRoutes.java
@@ -24,7 +24,6 @@ import javax.ws.rs.POST;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 
-import org.apache.james.core.Username;
 import org.apache.james.mailbox.MessageUid;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.exception.MailboxNotFoundException;
@@ -53,7 +52,6 @@ import io.swagger.annotations.ApiImplicitParams;
 import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
-import spark.HaltException;
 import spark.Request;
 import spark.Route;
 import spark.Service;
@@ -61,11 +59,8 @@ import spark.Service;
 @Api(tags = "ReIndexing (mailboxes)")
 @Path("/mailboxes")
 @Produces("application/json")
-public class ReindexingRoutes implements Routes {
-
-
+public class MailboxesRoutes implements Routes {
     private static final String BASE_PATH = "/mailboxes";
-    private static final String USER_QUERY_PARAM = "user";
     private static final String RE_INDEX_FAILED_MESSAGES_QUERY_PARAM = "reIndexFailedMessagesOf";
     private static final String MAILBOX_PARAM = ":mailbox";
     private static final String UID_PARAM = ":uid";
@@ -81,7 +76,7 @@ public class ReindexingRoutes implements Routes {
     private final JsonTransformer jsonTransformer;
 
     @Inject
-    ReindexingRoutes(TaskManager taskManager, PreviousReIndexingService previousReIndexingService, MailboxId.Factory mailboxIdFactory, ReIndexer reIndexer, JsonTransformer jsonTransformer) {
+    MailboxesRoutes(TaskManager taskManager, PreviousReIndexingService previousReIndexingService, MailboxId.Factory mailboxIdFactory, ReIndexer reIndexer, JsonTransformer jsonTransformer) {
         this.taskManager = taskManager;
         this.previousReIndexingService = previousReIndexingService;
         this.mailboxIdFactory = mailboxIdFactory;
@@ -114,13 +109,6 @@ public class ReindexingRoutes implements Routes {
             example = "?task=reIndex",
             value = "Compulsory. Only supported value is `reIndex`"),
         @ApiImplicitParam(
-            name = "user",
-            paramType = "query parameter",
-            dataType = "String",
-            defaultValue = "none",
-            example = "?user=toto%40domain.tld",
-            value = "optional. If present, only mailboxes of that user will be reIndexed."),
-        @ApiImplicitParam(
             name = "reIndexFailedMessagesOf",
             paramType = "query parameter",
             dataType = "String",
@@ -142,14 +130,7 @@ public class ReindexingRoutes implements Routes {
     }
 
     private Task reIndexAll(Request request) throws MailboxException {
-        boolean userReIndexing = !Strings.isNullOrEmpty(request.queryParams(USER_QUERY_PARAM));
         boolean indexingCorrection = !Strings.isNullOrEmpty(request.queryParams(RE_INDEX_FAILED_MESSAGES_QUERY_PARAM));
-        if (userReIndexing && indexingCorrection) {
-            throw rejectInvalidQueryParameterCombination();
-        }
-        if (userReIndexing) {
-            return reIndexer.reIndex(extractUser(request));
-        }
         if (indexingCorrection) {
             IndexingDetailInformation indexingDetailInformation = retrieveIndexingExecutionDetails(request);
             return reIndexer.reIndex(indexingDetailInformation.failures());
@@ -192,14 +173,6 @@ public class ReindexingRoutes implements Routes {
         }
     }
 
-    private HaltException rejectInvalidQueryParameterCombination() {
-        return ErrorResponder.builder()
-            .statusCode(HttpStatus.BAD_REQUEST_400)
-            .type(ErrorResponder.ErrorType.INVALID_ARGUMENT)
-            .message("Can not specify '" + USER_QUERY_PARAM + "' and '" + RE_INDEX_FAILED_MESSAGES_QUERY_PARAM + "' query parameters at the same time")
-            .haltError();
-    }
-
     @POST
     @Path("/{mailboxId}")
     @ApiOperation(value = "Re-indexes all the mails in a mailbox")
@@ -289,19 +262,6 @@ public class ReindexingRoutes implements Routes {
         };
     }
 
-    private Username extractUser(Request request) {
-        try {
-            return Username.of(request.queryParams(USER_QUERY_PARAM));
-        } catch (Exception e) {
-            throw ErrorResponder.builder()
-                .statusCode(HttpStatus.BAD_REQUEST_400)
-                .type(ErrorResponder.ErrorType.INVALID_ARGUMENT)
-                .message("Error while parsing 'user'")
-                .cause(e)
-                .haltError();
-        }
-    }
-
     private MailboxId extractMailboxId(Request request) {
         try {
             return mailboxIdFactory.fromString(request.params(MAILBOX_PARAM));
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessageIdReindexingRoutes.java b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessagesRoutes.java
similarity index 92%
rename from server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessageIdReindexingRoutes.java
rename to server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessagesRoutes.java
index a283262..45d235f 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessageIdReindexingRoutes.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/MessagesRoutes.java
@@ -19,7 +19,7 @@
 
 package org.apache.james.webadmin.routes;
 
-import static org.apache.james.webadmin.routes.ReindexingRoutes.TASK_PARAMETER;
+import static org.apache.james.webadmin.routes.MailboxesRoutes.TASK_PARAMETER;
 
 import javax.inject.Inject;
 import javax.ws.rs.POST;
@@ -49,7 +49,7 @@ import spark.Service;
 @Api(tags = "MessageIdReIndexing")
 @Path("/messages")
 @Produces("application/json")
-public class MessageIdReindexingRoutes implements Routes {
+public class MessagesRoutes implements Routes {
     private static final String MESSAGE_ID_PARAM = ":messageId";
     private static final String BASE_PATH = "/messages";
     private static final String MESSAGE_PATH = BASE_PATH + "/" + MESSAGE_ID_PARAM;
@@ -60,7 +60,7 @@ public class MessageIdReindexingRoutes implements Routes {
     private final JsonTransformer jsonTransformer;
 
     @Inject
-    MessageIdReindexingRoutes(TaskManager taskManager, MessageId.Factory messageIdFactory, MessageIdReIndexer reIndexer, JsonTransformer jsonTransformer) {
+    MessagesRoutes(TaskManager taskManager, MessageId.Factory messageIdFactory, MessageIdReIndexer reIndexer, JsonTransformer jsonTransformer) {
         this.taskManager = taskManager;
         this.messageIdFactory = messageIdFactory;
         this.reIndexer = reIndexer;
@@ -105,7 +105,7 @@ public class MessageIdReindexingRoutes implements Routes {
     private Route reIndexMessage() {
         return TaskFromRequestRegistry.builder()
             .parameterName(TASK_PARAMETER)
-            .register(ReindexingRoutes.RE_INDEX, request -> reIndexer.reIndex(extractMessageId(request)))
+            .register(MailboxesRoutes.RE_INDEX, request -> reIndexer.reIndex(extractMessageId(request)))
             .buildAsRoute(taskManager);
     }
 
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/UserMailboxesRoutes.java b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/UserMailboxesRoutes.java
index d5e814e..9e040a0 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/UserMailboxesRoutes.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/main/java/org/apache/james/webadmin/routes/UserMailboxesRoutes.java
@@ -19,17 +19,26 @@
 
 package org.apache.james.webadmin.routes;
 
+import static org.apache.james.webadmin.routes.MailboxesRoutes.RE_INDEX;
+import static org.apache.james.webadmin.routes.MailboxesRoutes.TASK_PARAMETER;
+
 import javax.inject.Inject;
 import javax.ws.rs.DELETE;
 import javax.ws.rs.GET;
+import javax.ws.rs.POST;
 import javax.ws.rs.PUT;
 import javax.ws.rs.Path;
 import javax.ws.rs.Produces;
 
 import org.apache.james.core.Username;
+import org.apache.james.mailbox.indexer.ReIndexer;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.task.TaskManager;
 import org.apache.james.webadmin.Constants;
 import org.apache.james.webadmin.Routes;
 import org.apache.james.webadmin.service.UserMailboxesService;
+import org.apache.james.webadmin.tasks.TaskFromRequestRegistry;
+import org.apache.james.webadmin.tasks.TaskIdDto;
 import org.apache.james.webadmin.utils.ErrorResponder;
 import org.apache.james.webadmin.utils.ErrorResponder.ErrorType;
 import org.apache.james.webadmin.utils.JsonTransformer;
@@ -47,6 +56,7 @@ import io.swagger.annotations.ApiOperation;
 import io.swagger.annotations.ApiResponse;
 import io.swagger.annotations.ApiResponses;
 import spark.Request;
+import spark.Route;
 import spark.Service;
 
 @Api(tags = "User's Mailbox")
@@ -65,12 +75,18 @@ public class UserMailboxesRoutes implements Routes {
 
     private final UserMailboxesService userMailboxesService;
     private final JsonTransformer jsonTransformer;
+    private final TaskManager taskManager;
+    private final MailboxId.Factory mailboxIdFactory;
+    private final ReIndexer reIndexer;
     private Service service;
 
     @Inject
-    public UserMailboxesRoutes(UserMailboxesService userMailboxesService, JsonTransformer jsonTransformer) {
+    UserMailboxesRoutes(UserMailboxesService userMailboxesService, JsonTransformer jsonTransformer, TaskManager taskManager, MailboxId.Factory mailboxIdFactory, ReIndexer reIndexer) {
         this.userMailboxesService = userMailboxesService;
         this.jsonTransformer = jsonTransformer;
+        this.taskManager = taskManager;
+        this.mailboxIdFactory = mailboxIdFactory;
+        this.reIndexer = reIndexer;
     }
 
     @Override
@@ -91,6 +107,8 @@ public class UserMailboxesRoutes implements Routes {
         defineDeleteUserMailbox();
 
         defineDeleteUserMailboxes();
+
+        service.post(USER_MAILBOXES_BASE, defineReIndexMailboxes(), jsonTransformer);
     }
 
     @GET
@@ -121,6 +139,31 @@ public class UserMailboxesRoutes implements Routes {
         }, jsonTransformer);
     }
 
+    @POST
+    @ApiImplicitParams({
+        @ApiImplicitParam(required = true, dataType = "string", name = "username", paramType = "path"),
+        @ApiImplicitParam(
+            required = true,
+            name = "task",
+            paramType = "query parameter",
+            dataType = "String",
+            defaultValue = "none",
+            example = "?task=reIndex",
+            value = "Compulsory. Only supported value is `reIndex`")
+    })
+    @ApiOperation(value = "Perform an action on a user mailbox")
+    @ApiResponses(value = {
+        @ApiResponse(code = HttpStatus.CREATED_201, message = "Task is created", response = TaskIdDto.class),
+        @ApiResponse(code = HttpStatus.INTERNAL_SERVER_ERROR_500, message = "Internal server error - Something went bad on the server side."),
+        @ApiResponse(code = HttpStatus.BAD_REQUEST_400, message = "Bad request - details in the returned error message")
+    })
+    public Route defineReIndexMailboxes() {
+        return TaskFromRequestRegistry.builder()
+            .parameterName(TASK_PARAMETER)
+            .register(RE_INDEX, request -> reIndexer.reIndex(getUsernameParam(request)))
+            .buildAsRoute(taskManager);
+    }
+
     @DELETE
     @Path("/{mailboxName}")
     @ApiImplicitParams({
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/ReindexingRoutesTest.java b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MailboxesRoutesTest.java
similarity index 71%
rename from server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/ReindexingRoutesTest.java
rename to server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MailboxesRoutesTest.java
index 40ab5de..67b0043 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/ReindexingRoutesTest.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MailboxesRoutesTest.java
@@ -39,7 +39,6 @@ import org.apache.james.mailbox.MessageManager;
 import org.apache.james.mailbox.indexer.ReIndexer;
 import org.apache.james.mailbox.inmemory.InMemoryId;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxManager;
-import org.apache.james.mailbox.inmemory.InMemoryMessageId;
 import org.apache.james.mailbox.inmemory.manager.InMemoryIntegrationResources;
 import org.apache.james.mailbox.model.ComposedMessageId;
 import org.apache.james.mailbox.model.Mailbox;
@@ -55,13 +54,10 @@ import org.apache.james.webadmin.service.PreviousReIndexingService;
 import org.apache.james.webadmin.utils.ErrorResponder;
 import org.apache.james.webadmin.utils.JsonTransformer;
 import org.apache.mailbox.tools.indexer.FullReindexingTask;
-import org.apache.mailbox.tools.indexer.MessageIdReIndexerImpl;
-import org.apache.mailbox.tools.indexer.MessageIdReIndexingTask;
 import org.apache.mailbox.tools.indexer.ReIndexerImpl;
 import org.apache.mailbox.tools.indexer.ReIndexerPerformer;
 import org.apache.mailbox.tools.indexer.SingleMailboxReindexingTask;
 import org.apache.mailbox.tools.indexer.SingleMessageReindexingTask;
-import org.apache.mailbox.tools.indexer.UserReindexingTask;
 import org.eclipse.jetty.http.HttpStatus;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
@@ -71,7 +67,7 @@ import org.mockito.ArgumentCaptor;
 
 import io.restassured.RestAssured;
 
-class ReindexingRoutesTest {
+class MailboxesRoutesTest {
     private static final Username USERNAME = Username.of("benwa@apache.org");
     private static final MailboxPath INBOX = MailboxPath.inbox(USERNAME);
 
@@ -97,15 +93,11 @@ class ReindexingRoutesTest {
 
         webAdminServer = WebAdminUtils.createWebAdminServer(
                 new TasksRoutes(taskManager, jsonTransformer),
-                new ReindexingRoutes(
+                new MailboxesRoutes(
                     taskManager,
                     new PreviousReIndexingService(taskManager),
                     mailboxIdFactory,
                     reIndexer,
-                    jsonTransformer),
-                new MessageIdReindexingRoutes(taskManager,
-                    new InMemoryMessageId.Factory(),
-                    new MessageIdReIndexerImpl(reIndexerPerformer),
                     jsonTransformer))
             .start();
 
@@ -275,195 +267,6 @@ class ReindexingRoutesTest {
     }
 
     @Nested
-    class UserReprocessing {
-        @Nested
-        class Validation {
-            @Test
-            void userReprocessingShouldFailWithNoTask() {
-                given()
-                    .queryParam("user", USERNAME.asString())
-                .when()
-                    .post("/mailboxes")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Invalid arguments supplied in the user request"))
-                    .body("details", is("'task' query parameter is compulsory. Supported values are [reIndex]"));
-            }
-
-            @Test
-            void userReprocessingShouldFailWithBadTask() {
-                given()
-                    .queryParam("user", USERNAME.asString())
-                    .queryParam("task", "bad")
-                .when()
-                    .post("/mailboxes")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Invalid arguments supplied in the user request"))
-                    .body("details", is("Invalid value supplied for query parameter 'task': bad. Supported values are [reIndex]"));
-            }
-
-            @Test
-            void userReprocessingShouldFailWithBadUser() {
-                given()
-                    .queryParam("user", "bad@bad@bad")
-                    .queryParam("task", "reIndex")
-                .when()
-                    .post("/mailboxes")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Error while parsing 'user'"));
-            }
-        }
-
-        @Nested
-        class TaskDetails {
-            @Test
-            void userReprocessingShouldNotFailWhenNoMail() {
-                String taskId = given()
-                    .queryParam("user", USERNAME.asString())
-                    .queryParam("task", "reIndex")
-                .when()
-                    .post("/mailboxes")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"))
-                    .body("taskId", is(notNullValue()))
-                    .body("type", is(UserReindexingTask.USER_RE_INDEXING.asString()))
-                    .body("additionalInformation.username", is("benwa@apache.org"))
-                    .body("additionalInformation.successfullyReprocessedMailCount", is(0))
-                    .body("additionalInformation.failedReprocessedMailCount", is(0))
-                    .body("startedDate", is(notNullValue()))
-                    .body("submitDate", is(notNullValue()))
-                    .body("completedDate", is(notNullValue()));
-            }
-
-            @Test
-            void userReprocessingShouldReturnTaskDetailsWhenMail() throws Exception {
-                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
-                mailboxManager.createMailbox(INBOX, systemSession).get();
-                mailboxManager.getMailbox(INBOX, systemSession)
-                    .appendMessage(
-                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
-                        systemSession);
-
-                String taskId = given()
-                    .queryParam("user", USERNAME.asString())
-                    .queryParam("task", "reIndex")
-                .when()
-                    .post("/mailboxes")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"))
-                    .body("taskId", is(notNullValue()))
-                    .body("type", is(UserReindexingTask.USER_RE_INDEXING.asString()))
-                    .body("additionalInformation.username", is("benwa@apache.org"))
-                    .body("additionalInformation.successfullyReprocessedMailCount", is(1))
-                    .body("additionalInformation.failedReprocessedMailCount", is(0))
-                    .body("startedDate", is(notNullValue()))
-                    .body("submitDate", is(notNullValue()))
-                    .body("completedDate", is(notNullValue()));
-            }
-
-            @Test
-            void userReprocessingShouldReturnTaskDetailsWhenFailing() throws Exception {
-                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
-                MailboxId mailboxId = mailboxManager.createMailbox(INBOX, systemSession).get();
-                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
-                    .appendMessage(
-                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
-                        systemSession);
-
-                doThrow(new RuntimeException())
-                    .when(searchIndex)
-                    .add(any(MailboxSession.class), any(Mailbox.class), any(MailboxMessage.class));
-
-                String taskId = with()
-                    .queryParam("user", USERNAME.asString())
-                    .queryParam("task", "reIndex")
-                    .post("/mailboxes")
-                    .jsonPath()
-                    .get("taskId");
-
-                long uidAsLong = composedMessageId.getUid().asLong();
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("failed"))
-                    .body("taskId", is(notNullValue()))
-                    .body("type", is(UserReindexingTask.USER_RE_INDEXING.asString()))
-                    .body("additionalInformation.successfullyReprocessedMailCount", is(0))
-                    .body("additionalInformation.failedReprocessedMailCount", is(1))
-                    .body("additionalInformation.failures.\"" + mailboxId.serialize() + "\"[0].uid", is(Long.valueOf(uidAsLong).intValue()))
-                    .body("startedDate", is(notNullValue()))
-                    .body("submitDate", is(notNullValue()));
-            }
-        }
-
-        @Nested
-        class SideEffects {
-            @Test
-            void userReprocessingShouldPerformReprocessingWhenMail() throws Exception {
-                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
-                MailboxId mailboxId = mailboxManager.createMailbox(INBOX, systemSession).get();
-                ComposedMessageId createdMessage = mailboxManager.getMailbox(INBOX, systemSession)
-                    .appendMessage(
-                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
-                        systemSession);
-
-                String taskId = given()
-                    .queryParam("user", USERNAME.asString())
-                    .queryParam("task", "reIndex")
-                .when()
-                    .post("/mailboxes")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"));
-
-
-                ArgumentCaptor<MailboxMessage> messageCaptor = ArgumentCaptor.forClass(MailboxMessage.class);
-                ArgumentCaptor<MailboxId> mailboxIdCaptor = ArgumentCaptor.forClass(MailboxId.class);
-                ArgumentCaptor<Mailbox> mailboxCaptor2 = ArgumentCaptor.forClass(Mailbox.class);
-
-                verify(searchIndex).deleteAll(any(MailboxSession.class), mailboxIdCaptor.capture());
-                verify(searchIndex).add(any(MailboxSession.class), mailboxCaptor2.capture(), messageCaptor.capture());
-                verifyNoMoreInteractions(searchIndex);
-
-                assertThat(mailboxIdCaptor.getValue()).matches(capturedMailboxId -> capturedMailboxId.equals(mailboxId));
-                assertThat(mailboxCaptor2.getValue()).matches(mailbox -> mailbox.getMailboxId().equals(mailboxId));
-                assertThat(messageCaptor.getValue()).matches(message -> message.getMailboxId().equals(mailboxId)
-                    && message.getUid().equals(createdMessage.getUid()));
-            }
-        }
-    }
-
-    @Nested
     class MailboxReprocessing {
         @Nested
         class Validation {
@@ -818,137 +621,6 @@ class ReindexingRoutesTest {
     }
 
     @Nested
-    class MessageIdReprocessing {
-        @Nested
-        class Validation {
-            @Test
-            void messageIdReprocessingShouldFailWithNoTask() {
-                when()
-                    .post("/messages/7")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Invalid arguments supplied in the user request"))
-                    .body("details", is("'task' query parameter is compulsory. Supported values are [reIndex]"));
-            }
-
-            @Test
-            void messageIdReprocessingShouldFailWithBadTask() {
-                when()
-                    .post("/messages/7?task=bad")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Invalid arguments supplied in the user request"))
-                    .body("details", is("Invalid value supplied for query parameter 'task': bad. Supported values are [reIndex]"));
-            }
-
-            @Test
-            void messageIdReprocessingShouldFailWithBadMessageId() {
-                when()
-                    .post("/messages/bad?task=reIndex")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Error while parsing 'messageId'"));
-            }
-        }
-
-        @Nested
-        class TaskDetails {
-            @Test
-            void messageIdReprocessingShouldNotFailWhenUidNotFound() {
-                String taskId = when()
-                    .post("/messages/1?task=reIndex")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"))
-                    .body("taskId", is(notNullValue()))
-                    .body("type", is(MessageIdReIndexingTask.TYPE.asString()))
-                    .body("additionalInformation.messageId", is("1"))
-                    .body("startedDate", is(notNullValue()))
-                    .body("submitDate", is(notNullValue()))
-                    .body("completedDate", is(notNullValue()));
-            }
-
-            @Test
-            void messageIdReprocessingShouldReturnTaskDetailsWhenMail() throws Exception {
-                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
-                mailboxManager.createMailbox(INBOX, systemSession).get();
-                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
-                    .appendMessage(
-                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
-                        systemSession);
-
-                String taskId = when()
-                    .post("/messages/" + composedMessageId.getMessageId().serialize() + "?task=reIndex")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"))
-                    .body("taskId", is(notNullValue()))
-                    .body("type", is(MessageIdReIndexingTask.TYPE.asString()))
-                    .body("additionalInformation.messageId", is(composedMessageId.getMessageId().serialize()))
-                    .body("startedDate", is(notNullValue()))
-                    .body("submitDate", is(notNullValue()))
-                    .body("completedDate", is(notNullValue()));
-            }
-        }
-
-        @Nested
-        class SideEffects {
-            @Test
-            void messageIdReprocessingShouldPerformReprocessingWhenMail() throws Exception {
-                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
-                mailboxManager.createMailbox(INBOX, systemSession).get();
-                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
-                    .appendMessage(
-                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
-                        systemSession);
-
-                String taskId = when()
-                    .post("/messages/" + composedMessageId.getMessageId().serialize() + "?task=reIndex")
-                    .jsonPath()
-                    .get("taskId");
-
-                given()
-                    .basePath(TasksRoutes.BASE)
-                .when()
-                    .get(taskId + "/await")
-                .then()
-                    .body("status", is("completed"));
-
-
-                ArgumentCaptor<MailboxMessage> messageCaptor = ArgumentCaptor.forClass(MailboxMessage.class);
-                ArgumentCaptor<Mailbox> mailboxCaptor = ArgumentCaptor.forClass(Mailbox.class);
-
-                verify(searchIndex).add(any(MailboxSession.class), mailboxCaptor.capture(), messageCaptor.capture());
-                verifyNoMoreInteractions(searchIndex);
-
-                assertThat(mailboxCaptor.getValue()).matches(mailbox -> mailbox.getMailboxId().equals(composedMessageId.getMailboxId()));
-                assertThat(messageCaptor.getValue()).matches(message -> message.getComposedMessageIdWithMetaData()
-                    .getComposedMessageId()
-                    .getMessageId()
-                    .equals(composedMessageId.getMessageId()));
-            }
-        }
-    }
-
-    @Nested
     class FixingReIndexing {
         @Nested
         class Validation {
@@ -976,30 +648,6 @@ class ReindexingRoutesTest {
             }
 
             @Test
-            void fixingReIndexingShouldThrowOnUserParameter() {
-                String taskId = with()
-                    .post("/mailboxes?task=reIndex")
-                    .jsonPath()
-                    .get("taskId");
-
-                with()
-                    .basePath(TasksRoutes.BASE)
-                    .get(taskId + "/await");
-
-                given()
-                    .queryParam("reIndexFailedMessagesOf", taskId)
-                    .queryParam("task", "reIndex")
-                    .queryParam("user", "bob@domain.tld")
-                .when()
-                    .post("/mailboxes")
-                .then()
-                    .statusCode(HttpStatus.BAD_REQUEST_400)
-                    .body("statusCode", is(400))
-                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
-                    .body("message", is("Can not specify 'user' and 'reIndexFailedMessagesOf' query parameters at the same time"));
-            }
-
-            @Test
             void fixingReIndexingShouldFailWithBadTask() {
                 String taskId = with()
                     .post("/mailboxes?task=reIndex")
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MalformedUrlRoutesTest.java b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MalformedUrlRoutesTest.java
index c9b4e4a..1589fde 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MalformedUrlRoutesTest.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MalformedUrlRoutesTest.java
@@ -35,7 +35,11 @@ import java.nio.charset.StandardCharsets;
 
 import org.apache.james.core.Username;
 import org.apache.james.mailbox.MailboxManager;
+import org.apache.james.mailbox.indexer.ReIndexer;
+import org.apache.james.mailbox.inmemory.InMemoryId;
 import org.apache.james.mailbox.inmemory.manager.InMemoryIntegrationResources;
+import org.apache.james.task.Hostname;
+import org.apache.james.task.MemoryTaskManager;
 import org.apache.james.user.api.UsersRepository;
 import org.apache.james.webadmin.WebAdminServer;
 import org.apache.james.webadmin.WebAdminUtils;
@@ -57,8 +61,10 @@ class MalformedUrlRoutesTest {
         usersRepository = mock(UsersRepository.class);
         when(usersRepository.contains(USERNAME)).thenReturn(true);
 
+        MemoryTaskManager taskManager = new MemoryTaskManager(new Hostname("foo"));
         webAdminServer = WebAdminUtils.createWebAdminServer(
-            new UserMailboxesRoutes(new UserMailboxesService(mailboxManager, usersRepository), new JsonTransformer()))
+            new UserMailboxesRoutes(new UserMailboxesService(mailboxManager, usersRepository), new JsonTransformer(),
+                taskManager, new InMemoryId.Factory(), mock(ReIndexer.class)))
             .start();
     }
 
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MessageRoutesTest.java b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MessageRoutesTest.java
new file mode 100644
index 0000000..6d068fb
--- /dev/null
+++ b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/MessageRoutesTest.java
@@ -0,0 +1,235 @@
+/****************************************************************
+ * Licensed to the Apache Software Foundation (ASF) under one   *
+ * or more contributor license agreements.  See the NOTICE file *
+ * distributed with this work for additional information        *
+ * regarding copyright ownership.  The ASF licenses this file   *
+ * to you under the Apache License, Version 2.0 (the            *
+ * "License"); you may not use this file except in compliance   *
+ * with the License.  You may obtain a copy of the License at   *
+ *                                                              *
+ *   http://www.apache.org/licenses/LICENSE-2.0                 *
+ *                                                              *
+ * Unless required by applicable law or agreed to in writing,   *
+ * software distributed under the License is distributed on an  *
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY       *
+ * KIND, either express or implied.  See the License for the    *
+ * specific language governing permissions and limitations      *
+ * under the License.                                           *
+ ****************************************************************/
+
+package org.apache.james.webadmin.routes;
+
+import static io.restassured.RestAssured.given;
+import static io.restassured.RestAssured.when;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
+
+import org.apache.james.core.Username;
+import org.apache.james.mailbox.MailboxSession;
+import org.apache.james.mailbox.MessageManager;
+import org.apache.james.mailbox.indexer.ReIndexer;
+import org.apache.james.mailbox.inmemory.InMemoryId;
+import org.apache.james.mailbox.inmemory.InMemoryMailboxManager;
+import org.apache.james.mailbox.inmemory.InMemoryMessageId;
+import org.apache.james.mailbox.inmemory.manager.InMemoryIntegrationResources;
+import org.apache.james.mailbox.model.ComposedMessageId;
+import org.apache.james.mailbox.model.Mailbox;
+import org.apache.james.mailbox.model.MailboxPath;
+import org.apache.james.mailbox.store.mail.model.MailboxMessage;
+import org.apache.james.mailbox.store.search.ListeningMessageSearchIndex;
+import org.apache.james.task.Hostname;
+import org.apache.james.task.MemoryTaskManager;
+import org.apache.james.webadmin.WebAdminServer;
+import org.apache.james.webadmin.WebAdminUtils;
+import org.apache.james.webadmin.utils.ErrorResponder;
+import org.apache.james.webadmin.utils.JsonTransformer;
+import org.apache.mailbox.tools.indexer.MessageIdReIndexerImpl;
+import org.apache.mailbox.tools.indexer.MessageIdReIndexingTask;
+import org.apache.mailbox.tools.indexer.ReIndexerImpl;
+import org.apache.mailbox.tools.indexer.ReIndexerPerformer;
+import org.eclipse.jetty.http.HttpStatus;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Nested;
+import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
+
+import io.restassured.RestAssured;
+
+class MessageRoutesTest {
+    private static final Username USERNAME = Username.of("benwa@apache.org");
+    private static final MailboxPath INBOX = MailboxPath.inbox(USERNAME);
+
+    private WebAdminServer webAdminServer;
+    private ListeningMessageSearchIndex searchIndex;
+    private InMemoryMailboxManager mailboxManager;
+
+    @BeforeEach
+    void beforeEach() {
+        mailboxManager = InMemoryIntegrationResources.defaultResources().getMailboxManager();
+        MemoryTaskManager taskManager = new MemoryTaskManager(new Hostname("foo"));
+        InMemoryId.Factory mailboxIdFactory = new InMemoryId.Factory();
+        searchIndex = mock(ListeningMessageSearchIndex.class);
+        ReIndexerPerformer reIndexerPerformer = new ReIndexerPerformer(
+            mailboxManager,
+            searchIndex,
+            mailboxManager.getMapperFactory());
+        ReIndexer reIndexer = new ReIndexerImpl(
+            reIndexerPerformer,
+            mailboxManager,
+            mailboxManager.getMapperFactory());
+        JsonTransformer jsonTransformer = new JsonTransformer();
+
+        webAdminServer = WebAdminUtils.createWebAdminServer(
+                new TasksRoutes(taskManager, jsonTransformer),
+                new MessagesRoutes(taskManager,
+                    new InMemoryMessageId.Factory(),
+                    new MessageIdReIndexerImpl(reIndexerPerformer),
+                    jsonTransformer))
+            .start();
+
+        RestAssured.requestSpecification = WebAdminUtils.buildRequestSpecification(webAdminServer).build();
+        RestAssured.enableLoggingOfRequestAndResponseIfValidationFails();
+    }
+
+    @AfterEach
+    void tearDown() {
+        webAdminServer.destroy();
+    }
+
+    @Nested
+    class MessageReprocessing {
+        @Nested
+        class Validation {
+            @Test
+            void messageIdReprocessingShouldFailWithNoTask() {
+                when()
+                    .post("/messages/7")
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", is(400))
+                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", is("Invalid arguments supplied in the user request"))
+                    .body("details", is("'task' query parameter is compulsory. Supported values are [reIndex]"));
+            }
+
+            @Test
+            void messageIdReprocessingShouldFailWithBadTask() {
+                when()
+                    .post("/messages/7?task=bad")
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", is(400))
+                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", is("Invalid arguments supplied in the user request"))
+                    .body("details", is("Invalid value supplied for query parameter 'task': bad. Supported values are [reIndex]"));
+            }
+
+            @Test
+            void messageIdReprocessingShouldFailWithBadMessageId() {
+                when()
+                    .post("/messages/bad?task=reIndex")
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", is(400))
+                    .body("type", is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", is("Error while parsing 'messageId'"));
+            }
+        }
+
+        @Nested
+        class TaskDetails {
+            @Test
+            void messageIdReprocessingShouldNotFailWhenUidNotFound() {
+                String taskId = when()
+                    .post("/messages/1?task=reIndex")
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", is("completed"))
+                    .body("taskId", is(notNullValue()))
+                    .body("type", is(MessageIdReIndexingTask.TYPE.asString()))
+                    .body("additionalInformation.messageId", is("1"))
+                    .body("startedDate", is(notNullValue()))
+                    .body("submitDate", is(notNullValue()))
+                    .body("completedDate", is(notNullValue()));
+            }
+
+            @Test
+            void messageIdReprocessingShouldReturnTaskDetailsWhenMail() throws Exception {
+                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
+                mailboxManager.createMailbox(INBOX, systemSession).get();
+                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
+                    .appendMessage(
+                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
+                        systemSession);
+
+                String taskId = when()
+                    .post("/messages/" + composedMessageId.getMessageId().serialize() + "?task=reIndex")
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", is("completed"))
+                    .body("taskId", is(notNullValue()))
+                    .body("type", is(MessageIdReIndexingTask.TYPE.asString()))
+                    .body("additionalInformation.messageId", is(composedMessageId.getMessageId().serialize()))
+                    .body("startedDate", is(notNullValue()))
+                    .body("submitDate", is(notNullValue()))
+                    .body("completedDate", is(notNullValue()));
+            }
+        }
+
+        @Nested
+        class SideEffects {
+            @Test
+            void messageIdReprocessingShouldPerformReprocessingWhenMail() throws Exception {
+                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
+                mailboxManager.createMailbox(INBOX, systemSession).get();
+                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
+                    .appendMessage(
+                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
+                        systemSession);
+
+                String taskId = when()
+                    .post("/messages/" + composedMessageId.getMessageId().serialize() + "?task=reIndex")
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", is("completed"));
+
+
+                ArgumentCaptor<MailboxMessage> messageCaptor = ArgumentCaptor.forClass(MailboxMessage.class);
+                ArgumentCaptor<Mailbox> mailboxCaptor = ArgumentCaptor.forClass(Mailbox.class);
+
+                verify(searchIndex).add(any(MailboxSession.class), mailboxCaptor.capture(), messageCaptor.capture());
+                verifyNoMoreInteractions(searchIndex);
+
+                assertThat(mailboxCaptor.getValue()).matches(mailbox -> mailbox.getMailboxId().equals(composedMessageId.getMailboxId()));
+                assertThat(messageCaptor.getValue()).matches(message -> message.getComposedMessageIdWithMetaData()
+                    .getComposedMessageId()
+                    .getMessageId()
+                    .equals(composedMessageId.getMessageId()));
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/UserMailboxesRoutesTest.java b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/UserMailboxesRoutesTest.java
index dd0d7bf..b33bafb 100644
--- a/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/UserMailboxesRoutesTest.java
+++ b/server/protocols/webadmin/webadmin-mailbox/src/test/java/org/apache/james/webadmin/routes/UserMailboxesRoutesTest.java
@@ -19,46 +19,64 @@
 
 package org.apache.james.webadmin.routes;
 
+import static io.restassured.RestAssured.given;
 import static io.restassured.RestAssured.when;
 import static io.restassured.RestAssured.with;
 import static org.apache.james.webadmin.Constants.SEPARATOR;
 import static org.apache.james.webadmin.routes.UserMailboxesRoutes.USERS_BASE;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.Matchers.notNullValue;
 import static org.hamcrest.collection.IsCollectionWithSize.hasSize;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyNoMoreInteractions;
 import static org.mockito.Mockito.when;
 
 import java.util.List;
 import java.util.Map;
 
 import org.apache.james.core.Username;
-import org.apache.james.mailbox.MailboxManager;
+import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.MailboxSessionUtil;
+import org.apache.james.mailbox.MessageManager;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.exception.MailboxExistsException;
 import org.apache.james.mailbox.exception.MailboxNotFoundException;
+import org.apache.james.mailbox.indexer.ReIndexer;
 import org.apache.james.mailbox.inmemory.InMemoryId;
+import org.apache.james.mailbox.inmemory.InMemoryMailboxManager;
 import org.apache.james.mailbox.inmemory.manager.InMemoryIntegrationResources;
+import org.apache.james.mailbox.model.ComposedMessageId;
+import org.apache.james.mailbox.model.Mailbox;
 import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.model.MailboxMetaData;
 import org.apache.james.mailbox.model.MailboxPath;
 import org.apache.james.mailbox.model.search.MailboxQuery;
+import org.apache.james.mailbox.store.mail.model.MailboxMessage;
+import org.apache.james.mailbox.store.search.ListeningMessageSearchIndex;
+import org.apache.james.task.Hostname;
+import org.apache.james.task.MemoryTaskManager;
 import org.apache.james.user.api.UsersRepository;
 import org.apache.james.webadmin.WebAdminServer;
 import org.apache.james.webadmin.WebAdminUtils;
 import org.apache.james.webadmin.service.UserMailboxesService;
+import org.apache.james.webadmin.utils.ErrorResponder;
 import org.apache.james.webadmin.utils.JsonTransformer;
+import org.apache.mailbox.tools.indexer.ReIndexerImpl;
+import org.apache.mailbox.tools.indexer.ReIndexerPerformer;
+import org.apache.mailbox.tools.indexer.UserReindexingTask;
 import org.eclipse.jetty.http.HttpStatus;
+import org.hamcrest.Matchers;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Nested;
 import org.junit.jupiter.api.Test;
+import org.mockito.ArgumentCaptor;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 
 import io.restassured.RestAssured;
 import io.restassured.http.ContentType;
@@ -66,16 +84,33 @@ import io.restassured.http.ContentType;
 class UserMailboxesRoutesTest {
     private static final Username USERNAME = Username.of("username");
     private static final String MAILBOX_NAME = "myMailboxName";
+    private static final MailboxPath INBOX = MailboxPath.inbox(USERNAME);
 
     private WebAdminServer webAdminServer;
     private UsersRepository usersRepository;
+    private ListeningMessageSearchIndex searchIndex;
 
-    private void createServer(MailboxManager mailboxManager) throws Exception {
+    private void createServer(InMemoryMailboxManager mailboxManager) throws Exception {
         usersRepository = mock(UsersRepository.class);
         when(usersRepository.contains(USERNAME)).thenReturn(true);
 
+
+        MemoryTaskManager taskManager = new MemoryTaskManager(new Hostname("foo"));
+        InMemoryId.Factory mailboxIdFactory = new InMemoryId.Factory();
+        searchIndex = mock(ListeningMessageSearchIndex.class);
+        ReIndexerPerformer reIndexerPerformer = new ReIndexerPerformer(
+            mailboxManager,
+            searchIndex,
+            mailboxManager.getMapperFactory());
+        ReIndexer reIndexer = new ReIndexerImpl(
+            reIndexerPerformer,
+            mailboxManager,
+            mailboxManager.getMapperFactory());
+
         webAdminServer = WebAdminUtils.createWebAdminServer(
-                new UserMailboxesRoutes(new UserMailboxesService(mailboxManager, usersRepository), new JsonTransformer()))
+                new UserMailboxesRoutes(new UserMailboxesService(mailboxManager, usersRepository), new JsonTransformer(),
+                    taskManager, mailboxIdFactory, reIndexer),
+                new TasksRoutes(taskManager, new JsonTransformer()))
             .start();
 
         RestAssured.requestSpecification = WebAdminUtils.buildRequestSpecification(webAdminServer)
@@ -734,12 +769,11 @@ class UserMailboxesRoutesTest {
 
     @Nested
     class ExceptionHandling {
-
-        private MailboxManager mailboxManager;
+        private InMemoryMailboxManager mailboxManager;
 
         @BeforeEach
         void setUp() throws Exception {
-            mailboxManager = mock(MailboxManager.class);
+            mailboxManager = mock(InMemoryMailboxManager.class);
             when(mailboxManager.createSystemSession(any())).thenReturn(MailboxSessionUtil.create(USERNAME));
 
             createServer(mailboxManager);
@@ -992,4 +1026,199 @@ class UserMailboxesRoutesTest {
 
     }
 
+    @Nested
+    class UserReprocessing {
+
+        private InMemoryMailboxManager mailboxManager;
+
+        @BeforeEach
+        void setUp() throws Exception {
+            mailboxManager = InMemoryIntegrationResources.defaultResources().getMailboxManager();
+            createServer(mailboxManager);
+        }
+
+        @Nested
+        class Validation {
+            @Test
+            void userReprocessingShouldFailWithNoTask() {
+                when()
+                    .post()
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", Matchers.is(400))
+                    .body("type", Matchers.is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", Matchers.is("Invalid arguments supplied in the user request"))
+                    .body("details", Matchers.is("'task' query parameter is compulsory. Supported values are [reIndex]"));
+            }
+
+            @Test
+            void userReprocessingShouldFailWithBadTask() {
+                given()
+                    .queryParam("task", "bad")
+                .when()
+                    .post()
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", Matchers.is(400))
+                    .body("type", Matchers.is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", Matchers.is("Invalid arguments supplied in the user request"))
+                    .body("details", Matchers.is("Invalid value supplied for query parameter 'task': bad. Supported values are [reIndex]"));
+            }
+
+            @Test
+            void userReprocessingShouldFailWithBadUser() {
+                RestAssured.requestSpecification = WebAdminUtils.buildRequestSpecification(webAdminServer)
+                    .setBasePath(USERS_BASE + SEPARATOR + "bad@bad@bad" + SEPARATOR + UserMailboxesRoutes.MAILBOXES)
+                    .build();
+
+                given()
+                    .queryParam("user", "bad@bad@bad")
+                    .queryParam("task", "reIndex")
+                .when()
+                    .post()
+                .then()
+                    .statusCode(HttpStatus.BAD_REQUEST_400)
+                    .body("statusCode", Matchers.is(400))
+                    .body("type", Matchers.is(ErrorResponder.ErrorType.INVALID_ARGUMENT.getType()))
+                    .body("message", Matchers.is("Invalid arguments supplied in the user request"))
+                    .body("details", Matchers.is("The username should not contain multiple domain delimiter."));
+            }
+        }
+
+        @Nested
+        class TaskDetails {
+            @Test
+            void userReprocessingShouldNotFailWhenNoMail() {
+                String taskId = given()
+                    .queryParam("task", "reIndex")
+                .when()
+                    .post()
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                    .when()
+                    .get(taskId + "/await")
+                    .then()
+                    .body("status", Matchers.is("completed"))
+                    .body("taskId", Matchers.is(notNullValue()))
+                    .body("type", Matchers.is(UserReindexingTask.USER_RE_INDEXING.asString()))
+                    .body("additionalInformation.username", Matchers.is("username"))
+                    .body("additionalInformation.successfullyReprocessedMailCount", Matchers.is(0))
+                    .body("additionalInformation.failedReprocessedMailCount", Matchers.is(0))
+                    .body("startedDate", Matchers.is(notNullValue()))
+                    .body("submitDate", Matchers.is(notNullValue()))
+                    .body("completedDate", Matchers.is(notNullValue()));
+            }
+
+            @Test
+            void userReprocessingShouldReturnTaskDetailsWhenMail() throws Exception {
+                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
+                mailboxManager.createMailbox(INBOX, systemSession).get();
+                mailboxManager.getMailbox(INBOX, systemSession)
+                    .appendMessage(
+                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
+                        systemSession);
+
+                String taskId = given()
+                    .queryParam("task", "reIndex")
+                .when()
+                    .post()
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", Matchers.is("completed"))
+                    .body("taskId", Matchers.is(notNullValue()))
+                    .body("type", Matchers.is(UserReindexingTask.USER_RE_INDEXING.asString()))
+                    .body("additionalInformation.username", Matchers.is("username"))
+                    .body("additionalInformation.successfullyReprocessedMailCount", Matchers.is(1))
+                    .body("additionalInformation.failedReprocessedMailCount", Matchers.is(0))
+                    .body("startedDate", Matchers.is(notNullValue()))
+                    .body("submitDate", Matchers.is(notNullValue()))
+                    .body("completedDate", Matchers.is(notNullValue()));
+            }
+
+            @Test
+            void userReprocessingShouldReturnTaskDetailsWhenFailing() throws Exception {
+                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
+                MailboxId mailboxId = mailboxManager.createMailbox(INBOX, systemSession).get();
+                ComposedMessageId composedMessageId = mailboxManager.getMailbox(INBOX, systemSession)
+                    .appendMessage(
+                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
+                        systemSession);
+
+                doThrow(new RuntimeException())
+                    .when(searchIndex)
+                    .add(any(MailboxSession.class), any(Mailbox.class), any(MailboxMessage.class));
+
+                String taskId = with()
+                    .queryParam("task", "reIndex")
+                .post()
+                    .jsonPath()
+                    .get("taskId");
+
+                long uidAsLong = composedMessageId.getUid().asLong();
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", Matchers.is("failed"))
+                    .body("taskId", Matchers.is(notNullValue()))
+                    .body("type", Matchers.is(UserReindexingTask.USER_RE_INDEXING.asString()))
+                    .body("additionalInformation.successfullyReprocessedMailCount", Matchers.is(0))
+                    .body("additionalInformation.failedReprocessedMailCount", Matchers.is(1))
+                    .body("additionalInformation.failures.\"" + mailboxId.serialize() + "\"[0].uid", Matchers.is(Long.valueOf(uidAsLong).intValue()))
+                    .body("startedDate", Matchers.is(notNullValue()))
+                    .body("submitDate", Matchers.is(notNullValue()));
+            }
+        }
+
+        @Nested
+        class SideEffects {
+            @Test
+            void userReprocessingShouldPerformReprocessingWhenMail() throws Exception {
+                MailboxSession systemSession = mailboxManager.createSystemSession(USERNAME);
+                MailboxId mailboxId = mailboxManager.createMailbox(INBOX, systemSession).get();
+                ComposedMessageId createdMessage = mailboxManager.getMailbox(INBOX, systemSession)
+                    .appendMessage(
+                        MessageManager.AppendCommand.builder().build("header: value\r\n\r\nbody"),
+                        systemSession);
+
+                String taskId = given()
+                    .queryParam("task", "reIndex")
+                .when()
+                    .post()
+                    .jsonPath()
+                    .get("taskId");
+
+                given()
+                    .basePath(TasksRoutes.BASE)
+                .when()
+                    .get(taskId + "/await")
+                .then()
+                    .body("status", Matchers.is("completed"));
+
+
+                ArgumentCaptor<MailboxMessage> messageCaptor = ArgumentCaptor.forClass(MailboxMessage.class);
+                ArgumentCaptor<MailboxId> mailboxIdCaptor = ArgumentCaptor.forClass(MailboxId.class);
+                ArgumentCaptor<Mailbox> mailboxCaptor2 = ArgumentCaptor.forClass(Mailbox.class);
+
+                verify(searchIndex).deleteAll(any(MailboxSession.class), mailboxIdCaptor.capture());
+                verify(searchIndex).add(any(MailboxSession.class), mailboxCaptor2.capture(), messageCaptor.capture());
+                verifyNoMoreInteractions(searchIndex);
+
+                assertThat(mailboxIdCaptor.getValue()).matches(capturedMailboxId -> capturedMailboxId.equals(mailboxId));
+                assertThat(mailboxCaptor2.getValue()).matches(mailbox -> mailbox.getMailboxId().equals(mailboxId));
+                assertThat(messageCaptor.getValue()).matches(message -> message.getMailboxId().equals(mailboxId)
+                    && message.getUid().equals(createdMessage.getUid()));
+            }
+        }
+    }
 }
diff --git a/src/site/markdown/server/manage-webadmin.md b/src/site/markdown/server/manage-webadmin.md
index d339699..c3b3786 100644
--- a/src/site/markdown/server/manage-webadmin.md
+++ b/src/site/markdown/server/manage-webadmin.md
@@ -31,6 +31,8 @@ Finally, please note that in case of a malformed URL the 400 bad request respons
  - [HealthCheck](#HealthCheck)
  - [Administrating domains](#Administrating_domains)
  - [Administrating users](#Administrating_users)
+ - [Administrating mailboxes](#Administrating_mailboxes)
+ - [Administrating messages](#Administrating_messages)
  - [Administrating user mailboxes](#Administrating_user_mailboxes)
  - [Administrating quotas by users](#Administrating_quotas_by_users)
  - [Administrating quotas by domains](#Administrating_quotas_by_domains)
@@ -48,7 +50,6 @@ Finally, please note that in case of a malformed URL the 400 bad request respons
  - [Administrating mail queues](#Administrating_mail_queues)
  - [Administrating DLP Configuration](#Administrating_DLP_Configuration)
  - [Administrating Sieve quotas](#Administrating_Sieve_quotas)
- - [ReIndexing](#ReIndexing)
  - [Event Dead Letter](#Event_Dead_Letter)
  - [Deleted Messages Vault](#Deleted_Messages_Vault)
  - [Task management](#Task_management)
@@ -336,6 +337,183 @@ Response codes:
 
  - 200: The user name list was successfully retrieved
 
+## Administrating mailboxes
+
+### All mailboxes
+
+Several actions can be performed on the server mailboxes.
+
+Request pattern is:
+
+```
+curl -XPOST /mailboxes?action=XXX,...
+```
+
+[More details about endpoints returning a task](#Endpoints_returning_a_task).
+
+Response codes:
+
+ - 201: Success. Corresponding task id is returned.
+ - 400: Error in the request. Details can be found in the reported error.
+
+The kind of task scheduled depends on the action parameter. See below for details.
+
+#### ReIndexing action
+
+Be also aware of the limits of this API:
+
+Warning: During the re-indexing, the result of search operations might be altered.
+
+Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
+
+Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
+concurrent changes done during the reIndexing might be ignored.
+
+The following actions can be performed:
+ - [ReIndexing all mails](#ReIndexing_all_mails)
+ - [Fixing previously failed ReIndexing](#Fixing_previously_failed_ReIndexing)
+
+##### ReIndexing all mails
+
+```
+curl -XPOST http://ip:port/mailboxes?task=reIndex
+```
+
+Will schedule a task for reIndexing all the mails stored on this James server.
+
+The scheduled task will have the following type `full-reindexing` and the following `additionalInformation`:
+
+```
+{
+  "successfullyReprocessedMailCount":18,
+  "failedReprocessedMailCount": 3,
+  "failures": {
+    "mbx1": [{"uid": 35}, {"uid": 45}],
+    "mbx2": [{"uid": 38}]
+  }
+}
+```
+
+##### Fixing previously failed ReIndexing
+
+Given `bbdb69c9-082a-44b0-a85a-6e33e74287a5` being a taskId generated for a reIndexing tasks
+
+```
+curl -XPOST http://ip:port/mailboxes?task=reIndex&reIndexFailedMessagesOf=bbdb69c9-082a-44b0-a85a-6e33e74287a5
+```
+
+The scheduled task will have the following type `error-recovery-indexation` and the following `additionalInformation`:
+
+```
+{
+  "successfullyReprocessedMailCount":18,
+  "failedReprocessedMailCount": 3,
+  "failures": {
+    "mbx1": [{"uid": 35}, {"uid": 45}],
+    "mbx2": [{"uid": 38}]
+  }
+}
+```
+
+### Single mailbox
+
+#### ReIndexing a mailbox mails
+
+```
+curl -XPOST http://ip:port/mailboxes/{mailboxId}?task=reIndex
+```
+
+Will schedule a task for reIndexing all the mails in one mailbox.
+
+Note that 'mailboxId' path parameter needs to be a (implementation dependent) valid mailboxId.
+
+[More details about endpoints returning a task](#Endpoints_returning_a_task).
+
+Response codes:
+
+ - 201: Success. Corresponding task id is returned.
+ - 400: Error in the request. Details can be found in the reported error.
+
+The scheduled task will have the following type `mailbox-reindexing` and the following `additionalInformation`:
+
+```
+{
+  "mailboxId":"{mailboxId}",
+  "successfullyReprocessedMailCount":18,
+  "failedReprocessedMailCount": 3,
+  "failures": {
+    "mbx1": [{"uid": 35}, {"uid": 45}],
+    "mbx2": [{"uid": 38}]
+  }
+}
+```
+
+Warning: During the re-indexing, the result of search operations might be altered.
+
+Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
+
+Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
+concurrent changes done during the reIndexing might be ignored.
+
+#### ReIndexing a single mail
+
+```
+curl -XPOST http://ip:port/mailboxes/{mailboxId}/uid/36?task=reIndex
+```
+
+Will schedule a task for reIndexing a single email.
+
+Note that 'mailboxId' path parameter needs to be a (implementation dependent) valid mailboxId.
+
+[More details about endpoints returning a task](#Endpoints_returning_a_task).
+
+Response codes:
+
+ - 201: Success. Corresponding task id is returned.
+ - 400: Error in the request. Details can be found in the reported error.
+
+The scheduled task will have the following type `message-reindexing` and the following `additionalInformation`:
+
+```
+{
+  "mailboxId":"{mailboxId}",
+  "uid":18
+}
+```
+
+Warning: During the re-indexing, the result of search operations might be altered.
+
+Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
+
+## Administrating Messages
+
+### ReIndexing a single mail by messageId
+
+```
+curl -XPOST http://ip:port/messages/{messageId}?task=reIndex
+```
+
+Will schedule a task for reIndexing a single email in all the mailboxes containing it.
+
+Note that 'messageId' path parameter needs to be a (implementation dependent) valid messageId.
+
+[More details about endpoints returning a task](#Endpoints_returning_a_task).
+
+Response codes:
+
+ - 201: Success. Corresponding task id is returned.
+ - 400: Error in the request. Details can be found in the reported error.
+
+The scheduled task will have the following type `messageId-reindexing` and the following `additionalInformation`:
+
+```
+{
+  "messageId":"18"
+}
+```
+
+Warning: During the re-indexing, the result of search operations might be altered.
+
 ## Administrating user mailboxes
 
  - [Creating a mailbox](#Creating_a_mailbox)
@@ -343,6 +521,7 @@ Response codes:
  - [Testing existence of a mailbox](#Testing_existence_of_a_mailbox)
  - [Listing user mailboxes](#Listing_user_mailboxes)
  - [Deleting_user_mailboxes](#Deleting_user_mailboxes)
+ - [ReIndexing a user mails](#ReIndexing_a_user_mails)
 
 ### Creating a mailbox
 
@@ -427,6 +606,42 @@ Response codes:
  - 204: The user do not have mailboxes anymore
  - 404: The user name does not exist
 
+### ReIndexing a user mails
+ 
+```
+curl -XPOST http://ip:port/users/usernameToBeUsed/mailboxes?task=reIndex
+```
+
+Will schedule a task for reIndexing all the mails in "bob@domain.com" mailboxes (encoded above).
+ 
+[More details about endpoints returning a task](#Endpoints_returning_a_task).
+ 
+Response codes:
+ 
+ - 201: Success. Corresponding task id is returned.
+ - 400: Error in the request. Details can be found in the reported error.
+
+The scheduled task will have the following type `user-reindexing` and the following `additionalInformation`:
+
+```
+{
+  "user":"bob@domain.com",
+  "successfullyReprocessedMailCount":18,
+  "failedReprocessedMailCount": 3,
+  "failures": {
+    "mbx1": [{"uid": 35}, {"uid": 45}],
+    "mbx2": [{"uid": 38}]
+  }
+}
+```
+
+Warning: During the re-indexing, the result of search operations might be altered.
+
+Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
+
+Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
+concurrent changes done during the reIndexing might be ignored.
+
 ## Administrating quotas by users
 
  - [Getting the quota for a user](#Getting_the_quota_for_a_user)
@@ -2500,220 +2715,6 @@ curl -XDELETE http://ip:port/sieve/quota/users/user@domain.com
 Response codes:
  - 204: Operation succeeded
 
-
-## ReIndexing
-
- - [ReIndexing all mails](#ReIndexing_all_mails)
- - [ReIndexing a user mails](#ReIndexing_a_user_mails)
- - [ReIndexing a mailbox mails](#ReIndexing_a_mailbox_mails)
- - [ReIndexing a single mail](#ReIndexing_a_single_mail)
- - [ReIndexing a single mail by messageId](#ReIndexing_a_single_mail_by_messageId)
-
-Be also aware of the limits of these APIs:
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
-Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
-
-Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
-concurrent changes done during the reIndexing might be ignored.
-
-### ReIndexing all mails
-
-```
-curl -XPOST http://ip:port/mailboxes?task=reIndex
-```
-
-Will schedule a task for reIndexing all the mails stored on this James server.
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `full-reindexing` and the following `additionalInformation`:
-
-```
-{
-  "successfullyReprocessedMailCount":18,
-  "failedReprocessedMailCount": 3,
-  "failures": {
-    "mbx1": [{"uid": 35}, {"uid": 45}],
-    "mbx2": [{"uid": 38}]
-  }
-}
-```
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
-Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
-
-Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
-concurrent changes done during the reIndexing might be ignored.
-
-### ReIndexing a user mails
-
-```
-curl -XPOST http://ip:port/mailboxes?task=reIndex,user=bob%40domain.com
-```
-
-Will schedule a task for reIndexing all the mails in "bob@domain.com" mailboxes (encoded above).
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `user-reindexing` and the following `additionalInformation`:
-
-```
-{
-  "user":"bob@domain.com",
-  "successfullyReprocessedMailCount":18,
-  "failedReprocessedMailCount": 3,
-  "failures": {
-    "mbx1": [{"uid": 35}, {"uid": 45}],
-    "mbx2": [{"uid": 38}]
-  }
-}
-```
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
-Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
-
-Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
-concurrent changes done during the reIndexing might be ignored.
-
-### ReIndexing a mailbox mails
-
-```
-curl -XPOST http://ip:port/mailboxes/{mailboxId}?task=reIndex
-```
-
-Will schedule a task for reIndexing all the mails in one mailbox.
-
-Note that 'mailboxId' path parameter needs to be a (implementation dependent) valid mailboxId.
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `mailbox-reindexing` and the following `additionalInformation`:
-
-```
-{
-  "mailboxId":"{mailboxId}",
-  "successfullyReprocessedMailCount":18,
-  "failedReprocessedMailCount": 3,
-  "failures": {
-    "mbx1": [{"uid": 35}, {"uid": 45}],
-    "mbx2": [{"uid": 38}]
-  }
-}
-```
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
-Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
-
-Warning: While we have been trying to reduce the inconsistency window to a maximum (by keeping track of ongoing events),
-concurrent changes done during the reIndexing might be ignored.
-
-### Fixing previously failed ReIndexing
-
-Given `bbdb69c9-082a-44b0-a85a-6e33e74287a5` being a taskId generated for a reIndexing tasks
-
-```
-curl -XPOST http://ip:port/mailboxes?task=reIndex&reIndexFailedMessagesOf=bbdb69c9-082a-44b0-a85a-6e33e74287a5
-```
-
-Will schedule a task for reIndexing all the mails that this task failed to reIndex.
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `error-recovery-indexation` and the following `additionalInformation`:
-
-```
-{
-  "successfullyReprocessedMailCount":18,
-  "failedReprocessedMailCount": 3,
-  "failures": {
-    "mbx1": [{"uid": 35}, {"uid": 45}],
-    "mbx2": [{"uid": 38}]
-  }
-}
-```
-
-### ReIndexing a single mail
-
-```
-curl -XPOST http://ip:port/mailboxes/{mailboxId}/uid/36?task=reIndex
-```
-
-Will schedule a task for reIndexing a single email.
-
-Note that 'mailboxId' path parameter needs to be a (implementation dependent) valid mailboxId.
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `message-reindexing` and the following `additionalInformation`:
-
-```
-{
-  "mailboxId":"{mailboxId}",
-  "uid":18
-}
-```
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
-Warning: Canceling this task should be considered unsafe as it will leave the currently reIndexed mailbox as partially indexed.
-
-### ReIndexing a single mail by messageId
-
-```
-curl -XPOST http://ip:port/messages/{messageId}?task=reIndex
-```
-
-Will schedule a task for reIndexing a single email in all the mailboxes containing it.
-
-Note that 'messageId' path parameter needs to be a (implementation dependent) valid messageId.
-
-[More details about endpoints returning a task](#Endpoints_returning_a_task).
-
-Response codes:
-
- - 201: Task generation succeeded. Corresponding task id is returned.
- - 400: Error in the request. Details can be found in the reported error.
-
-The scheduled task will have the following type `messageId-reindexing` and the following `additionalInformation`:
-
-```
-{
-  "messageId":"18"
-}
-```
-
-Warning: During the re-indexing, the result of search operations might be altered.
-
 ## Event Dead Letter
 
 The EventBus allows to register 'group listeners' that are called in a (potentially) distributed fashion. These group


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