You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@james.apache.org by bt...@apache.org on 2022/05/27 02:50:45 UTC

[james-project] branch master updated: JAMES-3768 Allow disabling Cassandra Mail queue view (#1009)

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


The following commit(s) were added to refs/heads/master by this push:
     new 35a62bff41 JAMES-3768 Allow disabling Cassandra Mail queue view (#1009)
35a62bff41 is described below

commit 35a62bff41e6d888261d51e1e93beaedf47b58dd
Author: Benoit TELLIER <bt...@linagora.com>
AuthorDate: Fri May 27 09:50:40 2022 +0700

    JAMES-3768 Allow disabling Cassandra Mail queue view (#1009)
---
 .../modules/ROOT/pages/configure/rabbitmq.adoc     |  5 ++
 .../sample-configuration/rabbitmq.properties       |  4 +
 .../james/CassandraRabbitMQJamesConfiguration.java | 23 +++++-
 .../james/CassandraRabbitMQJamesServerMain.java    |  2 +
 .../james/WithoutMailQueueViewImmutableTest.java   | 55 ++++++++++++++
 .../sample-configuration/rabbitmq.properties       |  4 +
 .../james/DistributedPOP3JamesConfiguration.java   | 18 ++++-
 .../james/DistributedPOP3JamesServerMain.java      |  2 +
 .../modules/mailbox/CassandraMailboxModule.java    | 23 +++++-
 .../rabbitmq/CassandraMailQueueViewModule.java     | 88 ++++++++++++++++++++++
 .../queue/rabbitmq/FakeMailQueueViewModule.java    | 35 +++++++++
 .../queue/rabbitmq/MailQueueViewChoice.java        | 67 ++++++++++++++++
 .../modules/queue/rabbitmq/RabbitMQModule.java     | 52 -------------
 .../queue/rabbitmq/view/api/FakeMailQueueView.java | 80 ++++++++++++++++++++
 src/site/xdoc/server/config-rabbitmq.xml           |  5 ++
 15 files changed, 406 insertions(+), 57 deletions(-)

diff --git a/server/apps/distributed-app/docs/modules/ROOT/pages/configure/rabbitmq.adoc b/server/apps/distributed-app/docs/modules/ROOT/pages/configure/rabbitmq.adoc
index 6b97764135..d07fb14a52 100644
--- a/server/apps/distributed-app/docs/modules/ROOT/pages/configure/rabbitmq.adoc
+++ b/server/apps/distributed-app/docs/modules/ROOT/pages/configure/rabbitmq.adoc
@@ -113,6 +113,11 @@ this implementation relies on Cassandra.
 |===
 | Property name | explanation
 
+| cassandra.view.enabled
+| Whether the Cassandra administrative view should be activated. Boolean value defaulting to true.
+Not necessarily needed for MDA deployments, mail queue management adds significant complexity.
+
+
 | mailqueue.view.sliceWindow
 | James divides the view into slices, each slice contains data for a given period, sliceWindow parameter controls this period.
 This dividing of periods allows faster browsing of the mail queue. Tips for choosing sliceWindow are explained in
diff --git a/server/apps/distributed-app/sample-configuration/rabbitmq.properties b/server/apps/distributed-app/sample-configuration/rabbitmq.properties
index b63acd9d85..3c196413bb 100644
--- a/server/apps/distributed-app/sample-configuration/rabbitmq.properties
+++ b/server/apps/distributed-app/sample-configuration/rabbitmq.properties
@@ -43,6 +43,10 @@ management.password=guest
 
 # Parameters for the Cassandra administrative view
 
+# Whether the Cassandra administrative view should be activated. Boolean value defaulting to true.
+# Not necessarily needed for MDA deployments, mail queue management adds significant complexity.
+# cassandra.view.enabled=true
+
 # Period of the window. Too large values will lead to wide rows while too little values might lead to many queries.
 # Use the number of mail per Cassandra row, along with your expected traffic, to determine this value
 # This value can only be decreased to a value dividing the current value
diff --git a/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesConfiguration.java b/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesConfiguration.java
index 738977bb04..1a07d32a91 100644
--- a/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesConfiguration.java
+++ b/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesConfiguration.java
@@ -26,6 +26,7 @@ import org.apache.james.data.UsersRepositoryModuleChooser;
 import org.apache.james.filesystem.api.FileSystem;
 import org.apache.james.filesystem.api.JamesDirectoriesProvider;
 import org.apache.james.modules.blobstore.BlobStoreConfiguration;
+import org.apache.james.modules.queue.rabbitmq.MailQueueViewChoice;
 import org.apache.james.server.core.JamesServerResourceLoader;
 import org.apache.james.server.core.MissingArgumentException;
 import org.apache.james.server.core.configuration.Configuration;
@@ -41,6 +42,7 @@ public class CassandraRabbitMQJamesConfiguration implements Configuration {
         private Optional<BlobStoreConfiguration> blobStoreConfiguration;
         private Optional<String> rootDirectory;
         private Optional<ConfigurationPath> configurationPath;
+        private Optional<MailQueueViewChoice> mailQueueViewChoice;
         private Optional<UsersRepositoryModuleChooser.Implementation> usersRepositoryImplementation;
 
         private Builder() {
@@ -49,6 +51,7 @@ public class CassandraRabbitMQJamesConfiguration implements Configuration {
             configurationPath = Optional.empty();
             blobStoreConfiguration = Optional.empty();
             usersRepositoryImplementation = Optional.empty();
+            mailQueueViewChoice = Optional.empty();
         }
 
         public Builder workingDirectory(String path) {
@@ -94,6 +97,11 @@ public class CassandraRabbitMQJamesConfiguration implements Configuration {
             return this;
         }
 
+        public Builder mailQueueViewChoice(MailQueueViewChoice mailQueueViewChoice) {
+            this.mailQueueViewChoice = Optional.of(mailQueueViewChoice);
+            return this;
+        }
+
         public CassandraRabbitMQJamesConfiguration build() {
             ConfigurationPath configurationPath = this.configurationPath.orElse(new ConfigurationPath(FileSystem.FILE_PROTOCOL_AND_CONF));
             JamesServerResourceLoader directories = new JamesServerResourceLoader(rootDirectory
@@ -115,12 +123,17 @@ public class CassandraRabbitMQJamesConfiguration implements Configuration {
             UsersRepositoryModuleChooser.Implementation usersRepositoryChoice = usersRepositoryImplementation.orElseGet(
                 () -> UsersRepositoryModuleChooser.Implementation.parse(configurationProvider));
 
+            MailQueueViewChoice mailQueueViewChoice = this.mailQueueViewChoice.orElseGet(Throwing.supplier(
+                () -> MailQueueViewChoice.parse(
+                    new PropertiesProvider(fileSystem, configurationPath))));
+
             return new CassandraRabbitMQJamesConfiguration(
                 configurationPath,
                 directories,
                 blobStoreConfiguration,
                 searchConfiguration,
-                usersRepositoryChoice);
+                usersRepositoryChoice,
+                mailQueueViewChoice);
         }
     }
 
@@ -133,13 +146,19 @@ public class CassandraRabbitMQJamesConfiguration implements Configuration {
     private final BlobStoreConfiguration blobStoreConfiguration;
     private final SearchConfiguration searchConfiguration;
     private final UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation;
+    private final MailQueueViewChoice mailQueueViewChoice;
 
-    public CassandraRabbitMQJamesConfiguration(ConfigurationPath configurationPath, JamesDirectoriesProvider directories, BlobStoreConfiguration blobStoreConfiguration, SearchConfiguration searchConfiguration, UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation) {
+    public CassandraRabbitMQJamesConfiguration(ConfigurationPath configurationPath, JamesDirectoriesProvider directories, BlobStoreConfiguration blobStoreConfiguration, SearchConfiguration searchConfiguration, UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation, MailQueueViewChoice mailQueueViewChoice) {
         this.configurationPath = configurationPath;
         this.directories = directories;
         this.blobStoreConfiguration = blobStoreConfiguration;
         this.searchConfiguration = searchConfiguration;
         this.usersRepositoryImplementation = usersRepositoryImplementation;
+        this.mailQueueViewChoice = mailQueueViewChoice;
+    }
+
+    public MailQueueViewChoice getMailQueueViewChoice() {
+        return mailQueueViewChoice;
     }
 
     @Override
diff --git a/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesServerMain.java b/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesServerMain.java
index 65044d2852..fc59322be8 100644
--- a/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesServerMain.java
+++ b/server/apps/distributed-app/src/main/java/org/apache/james/CassandraRabbitMQJamesServerMain.java
@@ -59,6 +59,7 @@ import org.apache.james.modules.protocols.ManageSieveServerModule;
 import org.apache.james.modules.protocols.POP3ServerModule;
 import org.apache.james.modules.protocols.ProtocolHandlerModule;
 import org.apache.james.modules.protocols.SMTPServerModule;
+import org.apache.james.modules.queue.rabbitmq.MailQueueViewChoice;
 import org.apache.james.modules.queue.rabbitmq.RabbitMQModule;
 import org.apache.james.modules.server.DKIMMailetModule;
 import org.apache.james.modules.server.DLPRoutesModule;
@@ -188,6 +189,7 @@ public class CassandraRabbitMQJamesServerMain implements JamesServerMain {
 
         return GuiceJamesServer.forConfiguration(configuration)
             .combineWith(MODULES)
+            .combineWith(MailQueueViewChoice.ModuleChooser.choose(configuration.getMailQueueViewChoice()))
             .combineWith(BlobStoreModulesChooser.chooseModules(blobStoreConfiguration))
             .combineWith(BlobStoreCacheModulesChooser.chooseModules(blobStoreConfiguration))
             .combineWith(SearchModuleChooser.chooseModules(searchConfiguration))
diff --git a/server/apps/distributed-app/src/test/java/org/apache/james/WithoutMailQueueViewImmutableTest.java b/server/apps/distributed-app/src/test/java/org/apache/james/WithoutMailQueueViewImmutableTest.java
new file mode 100644
index 0000000000..22f1e971e6
--- /dev/null
+++ b/server/apps/distributed-app/src/test/java/org/apache/james/WithoutMailQueueViewImmutableTest.java
@@ -0,0 +1,55 @@
+/****************************************************************
+ * 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;
+
+import org.apache.james.jmap.draft.JmapJamesServerContract;
+import org.apache.james.modules.AwsS3BlobStoreExtension;
+import org.apache.james.modules.RabbitMQExtension;
+import org.apache.james.modules.TestJMAPServerModule;
+import org.apache.james.modules.blobstore.BlobStoreConfiguration;
+import org.apache.james.modules.queue.rabbitmq.MailQueueViewChoice;
+import org.junit.jupiter.api.extension.RegisterExtension;
+
+class WithoutMailQueueViewImmutableTest implements JmapJamesServerContract, JamesServerContract {
+    static JamesServerBuilder<CassandraRabbitMQJamesConfiguration> baseExtension() {
+        return new JamesServerBuilder<CassandraRabbitMQJamesConfiguration>(tmpDir ->
+            CassandraRabbitMQJamesConfiguration.builder()
+                .workingDirectory(tmpDir)
+                .configurationFromClasspath()
+                .blobStore(BlobStoreConfiguration.builder()
+                    .s3()
+                    .disableCache()
+                    .deduplication()
+                    .noCryptoConfig())
+                .mailQueueViewChoice(MailQueueViewChoice.NONE)
+                .searchConfiguration(SearchConfiguration.scanning())
+                .build())
+            .server(configuration -> CassandraRabbitMQJamesServerMain.createServer(configuration)
+                .overrideWith(new TestJMAPServerModule()))
+            .extension(new CassandraExtension())
+            .extension(new RabbitMQExtension())
+            .extension(new AwsS3BlobStoreExtension());
+    }
+
+    @RegisterExtension
+    static JamesServerExtension jamesServerExtension = baseExtension()
+        .lifeCycle(JamesServerExtension.Lifecycle.PER_CLASS)
+        .build();
+}
\ No newline at end of file
diff --git a/server/apps/distributed-pop3-app/sample-configuration/rabbitmq.properties b/server/apps/distributed-pop3-app/sample-configuration/rabbitmq.properties
index d587cb647e..c65652e64b 100644
--- a/server/apps/distributed-pop3-app/sample-configuration/rabbitmq.properties
+++ b/server/apps/distributed-pop3-app/sample-configuration/rabbitmq.properties
@@ -43,6 +43,10 @@ management.password=guest
 
 # Parameters for the Cassandra administrative view
 
+# Whether the Cassandra administrative view should be activated. Boolean value defaulting to true.
+# Not necessarily needed for MDA deployments, mail queue management adds significant complexity.
+# cassandra.view.enabled=true
+
 # Period of the window. Too large values will lead to wide rows while too little values might lead to many queries.
 # Use the number of mail per Cassandra row, along with your expected traffic, to determine this value
 # This value can only be decreased to a value dividing the current value
diff --git a/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesConfiguration.java b/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesConfiguration.java
index 8ced491299..b1e9e30e0d 100644
--- a/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesConfiguration.java
+++ b/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesConfiguration.java
@@ -26,6 +26,7 @@ import org.apache.james.data.UsersRepositoryModuleChooser;
 import org.apache.james.filesystem.api.FileSystem;
 import org.apache.james.filesystem.api.JamesDirectoriesProvider;
 import org.apache.james.modules.blobstore.BlobStoreConfiguration;
+import org.apache.james.modules.queue.rabbitmq.MailQueueViewChoice;
 import org.apache.james.server.core.JamesServerResourceLoader;
 import org.apache.james.server.core.MissingArgumentException;
 import org.apache.james.server.core.configuration.Configuration;
@@ -42,6 +43,7 @@ public class DistributedPOP3JamesConfiguration implements Configuration {
         private Optional<String> rootDirectory;
         private Optional<ConfigurationPath> configurationPath;
         private Optional<UsersRepositoryModuleChooser.Implementation> usersRepositoryImplementation;
+        private Optional<MailQueueViewChoice> mailQueueViewChoice;
 
         private Builder() {
             searchConfiguration = Optional.empty();
@@ -49,6 +51,7 @@ public class DistributedPOP3JamesConfiguration implements Configuration {
             configurationPath = Optional.empty();
             blobStoreConfiguration = Optional.empty();
             usersRepositoryImplementation = Optional.empty();
+            mailQueueViewChoice = Optional.empty();
         }
 
         public Builder workingDirectory(String path) {
@@ -115,12 +118,17 @@ public class DistributedPOP3JamesConfiguration implements Configuration {
             UsersRepositoryModuleChooser.Implementation usersRepositoryChoice = usersRepositoryImplementation.orElseGet(
                 () -> UsersRepositoryModuleChooser.Implementation.parse(configurationProvider));
 
+            MailQueueViewChoice mailQueueViewChoice = this.mailQueueViewChoice.orElseGet(Throwing.supplier(
+                () -> MailQueueViewChoice.parse(
+                    new PropertiesProvider(fileSystem, configurationPath))));
+
             return new DistributedPOP3JamesConfiguration(
                 configurationPath,
                 directories,
                 blobStoreConfiguration,
                 searchConfiguration,
-                usersRepositoryChoice);
+                usersRepositoryChoice,
+                mailQueueViewChoice);
         }
     }
 
@@ -133,13 +141,19 @@ public class DistributedPOP3JamesConfiguration implements Configuration {
     private final BlobStoreConfiguration blobStoreConfiguration;
     private final SearchConfiguration searchConfiguration;
     private final UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation;
+    private final MailQueueViewChoice mailQueueViewChoice;
 
-    public DistributedPOP3JamesConfiguration(ConfigurationPath configurationPath, JamesDirectoriesProvider directories, BlobStoreConfiguration blobStoreConfiguration, SearchConfiguration searchConfiguration, UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation) {
+    public DistributedPOP3JamesConfiguration(ConfigurationPath configurationPath, JamesDirectoriesProvider directories, BlobStoreConfiguration blobStoreConfiguration, SearchConfiguration searchConfiguration, UsersRepositoryModuleChooser.Implementation usersRepositoryImplementation, MailQueueViewChoice mailQueueViewChoice) {
         this.configurationPath = configurationPath;
         this.directories = directories;
         this.blobStoreConfiguration = blobStoreConfiguration;
         this.searchConfiguration = searchConfiguration;
         this.usersRepositoryImplementation = usersRepositoryImplementation;
+        this.mailQueueViewChoice = mailQueueViewChoice;
+    }
+
+    public MailQueueViewChoice getMailQueueViewChoice() {
+        return mailQueueViewChoice;
     }
 
     @Override
diff --git a/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesServerMain.java b/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesServerMain.java
index 23b745c679..c8fda632a7 100644
--- a/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesServerMain.java
+++ b/server/apps/distributed-pop3-app/src/main/java/org/apache/james/DistributedPOP3JamesServerMain.java
@@ -66,6 +66,7 @@ import org.apache.james.modules.protocols.ManageSieveServerModule;
 import org.apache.james.modules.protocols.POP3ServerModule;
 import org.apache.james.modules.protocols.ProtocolHandlerModule;
 import org.apache.james.modules.protocols.SMTPServerModule;
+import org.apache.james.modules.queue.rabbitmq.MailQueueViewChoice;
 import org.apache.james.modules.queue.rabbitmq.RabbitMQModule;
 import org.apache.james.modules.server.DKIMMailetModule;
 import org.apache.james.modules.server.DataRoutesModules;
@@ -183,6 +184,7 @@ public class DistributedPOP3JamesServerMain implements JamesServerMain {
 
         return GuiceJamesServer.forConfiguration(configuration)
             .combineWith(MODULES)
+            .combineWith(MailQueueViewChoice.ModuleChooser.choose(configuration.getMailQueueViewChoice()))
             .combineWith(BlobStoreModulesChooser.chooseModules(blobStoreConfiguration))
             .combineWith(BlobStoreCacheModulesChooser.chooseModules(blobStoreConfiguration))
             .combineWith(SearchModuleChooser.chooseModules(searchConfiguration))
diff --git a/server/container/guice/cassandra/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java b/server/container/guice/cassandra/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
index 908741f8b7..5f9212c644 100644
--- a/server/container/guice/cassandra/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
+++ b/server/container/guice/cassandra/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
@@ -25,9 +25,14 @@ import javax.inject.Singleton;
 import org.apache.james.adapter.mailbox.UserRepositoryAuthenticator;
 import org.apache.james.adapter.mailbox.UserRepositoryAuthorizator;
 import org.apache.james.backends.cassandra.components.CassandraModule;
+import org.apache.james.backends.cassandra.init.configuration.CassandraConsistenciesConfiguration;
+import org.apache.james.backends.cassandra.versions.CassandraSchemaVersionManager;
 import org.apache.james.blob.api.BlobReferenceSource;
 import org.apache.james.events.EventListener;
 import org.apache.james.eventsourcing.Event;
+import org.apache.james.eventsourcing.eventstore.cassandra.CassandraEventStore;
+import org.apache.james.eventsourcing.eventstore.cassandra.EventStoreDao;
+import org.apache.james.eventsourcing.eventstore.cassandra.JsonEventSerializer;
 import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTO;
 import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTOModule;
 import org.apache.james.jmap.api.change.EmailChangeRepository;
@@ -115,8 +120,10 @@ import org.apache.james.utils.MailboxManagerDefinition;
 import org.apache.mailbox.tools.indexer.MessageIdReIndexerImpl;
 import org.apache.mailbox.tools.indexer.ReIndexerImpl;
 
+import com.datastax.driver.core.Session;
 import com.google.inject.AbstractModule;
 import com.google.inject.Inject;
+import com.google.inject.Provides;
 import com.google.inject.Scopes;
 import com.google.inject.TypeLiteral;
 import com.google.inject.multibindings.Multibinder;
@@ -155,7 +162,6 @@ public class CassandraMailboxModule extends AbstractModule {
         bind(StoreRightManager.class).in(Scopes.SINGLETON);
         bind(SessionProviderImpl.class).in(Scopes.SINGLETON);
 
-        bind(CassandraACLMapper.class).in(Scopes.SINGLETON);
         bind(CassandraMailboxMapper.class).in(Scopes.SINGLETON);
 
         bind(CassandraId.Factory.class).in(Scopes.SINGLETON);
@@ -239,6 +245,21 @@ public class CassandraMailboxModule extends AbstractModule {
         Multibinder.newSetBinder(binder(), BlobReferenceSource.class)
             .addBinding().to(MessageBlobReferenceSource.class);
     }
+
+    @Provides
+    @Singleton
+    CassandraACLMapper aclMapper(CassandraACLMapper.StoreV1 storeV1,
+                                 CassandraUserMailboxRightsDAO userMailboxRightsDAO,
+                                 CassandraACLDAOV2 cassandraACLDAOV2,
+                                 Session session,
+                                 CassandraConsistenciesConfiguration cassandraConsistenciesConfiguration,
+                                 CassandraSchemaVersionManager cassandraSchemaVersionManager) {
+        return new CassandraACLMapper(storeV1,
+            new CassandraACLMapper.StoreV2(userMailboxRightsDAO, cassandraACLDAOV2,
+                new CassandraEventStore(new EventStoreDao(session, JsonEventSerializer.forModules(ACLModule.ACL_UPDATE).withoutNestedType(),
+                    cassandraConsistenciesConfiguration))),
+            cassandraSchemaVersionManager);
+    }
     
     @Singleton
     private static class CassandraMailboxManagerDefinition extends MailboxManagerDefinition {
diff --git a/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/CassandraMailQueueViewModule.java b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/CassandraMailQueueViewModule.java
new file mode 100644
index 0000000000..986110dc55
--- /dev/null
+++ b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/CassandraMailQueueViewModule.java
@@ -0,0 +1,88 @@
+/****************************************************************
+ * 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.modules.queue.rabbitmq;
+
+import javax.inject.Named;
+import javax.inject.Singleton;
+
+import org.apache.james.backends.cassandra.components.CassandraModule;
+import org.apache.james.blob.api.BlobReferenceSource;
+import org.apache.james.eventsourcing.Event;
+import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTO;
+import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTOModule;
+import org.apache.james.lifecycle.api.StartUpCheck;
+import org.apache.james.queue.rabbitmq.view.api.MailQueueView;
+import org.apache.james.queue.rabbitmq.view.cassandra.BrowseStartDAO;
+import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueBrowser;
+import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueMailDelete;
+import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueMailStore;
+import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueView;
+import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueViewStartUpCheck;
+import org.apache.james.queue.rabbitmq.view.cassandra.ContentStartDAO;
+import org.apache.james.queue.rabbitmq.view.cassandra.DeletedMailsDAO;
+import org.apache.james.queue.rabbitmq.view.cassandra.EnqueuedMailsDAO;
+import org.apache.james.queue.rabbitmq.view.cassandra.MailQueueViewBlobReferenceSource;
+import org.apache.james.queue.rabbitmq.view.cassandra.configuration.CassandraMailQueueViewConfiguration;
+import org.apache.james.queue.rabbitmq.view.cassandra.configuration.CassandraMailQueueViewConfigurationModule;
+import org.apache.james.queue.rabbitmq.view.cassandra.configuration.EventsourcingConfigurationManagement;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Provides;
+import com.google.inject.Scopes;
+import com.google.inject.TypeLiteral;
+import com.google.inject.multibindings.Multibinder;
+
+public class CassandraMailQueueViewModule extends AbstractModule {
+    public static final String RABBITMQ_CONFIGURATION_NAME = "rabbitmq";
+
+    @Override
+    protected void configure() {
+        bind(EnqueuedMailsDAO.class).in(Scopes.SINGLETON);
+        bind(DeletedMailsDAO.class).in(Scopes.SINGLETON);
+        bind(BrowseStartDAO.class).in(Scopes.SINGLETON);
+        bind(CassandraMailQueueBrowser.class).in(Scopes.SINGLETON);
+        bind(CassandraMailQueueMailDelete.class).in(Scopes.SINGLETON);
+        bind(CassandraMailQueueMailStore.class).in(Scopes.SINGLETON);
+        bind(ContentStartDAO.class).in(Scopes.SINGLETON);
+
+        Multibinder<CassandraModule> cassandraModuleBinder = Multibinder.newSetBinder(binder(), CassandraModule.class);
+        cassandraModuleBinder.addBinding().toInstance(org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueViewModule.MODULE);
+
+        bind(EventsourcingConfigurationManagement.class).in(Scopes.SINGLETON);
+        Multibinder<EventDTOModule<? extends Event, ? extends EventDTO>> eventDTOModuleBinder = Multibinder.newSetBinder(binder(), new TypeLiteral<EventDTOModule<? extends Event, ? extends EventDTO>>() {});
+        eventDTOModuleBinder.addBinding().toInstance(CassandraMailQueueViewConfigurationModule.MAIL_QUEUE_VIEW_CONFIGURATION);
+
+        Multibinder.newSetBinder(binder(), StartUpCheck.class).addBinding().to(CassandraMailQueueViewStartUpCheck.class);
+
+        Multibinder.newSetBinder(binder(), BlobReferenceSource.class)
+            .addBinding().to(MailQueueViewBlobReferenceSource.class);
+    }
+
+    @Provides
+    @Singleton
+    public MailQueueView.Factory provideMailQueueViewFactory(CassandraMailQueueView.Factory cassandraMailQueueViewFactory) {
+        return cassandraMailQueueViewFactory;
+    }
+
+    @Provides
+    @Singleton
+    private CassandraMailQueueViewConfiguration getMailQueueViewConfiguration(@Named(RABBITMQ_CONFIGURATION_NAME) org.apache.commons.configuration2.Configuration configuration) {
+        return CassandraMailQueueViewConfiguration.from(configuration);
+    }
+}
diff --git a/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/FakeMailQueueViewModule.java b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/FakeMailQueueViewModule.java
new file mode 100644
index 0000000000..b1656ab902
--- /dev/null
+++ b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/FakeMailQueueViewModule.java
@@ -0,0 +1,35 @@
+/****************************************************************
+ * 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.modules.queue.rabbitmq;
+
+import javax.inject.Singleton;
+
+import org.apache.james.queue.rabbitmq.view.api.FakeMailQueueView;
+import org.apache.james.queue.rabbitmq.view.api.MailQueueView;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Provides;
+
+public class FakeMailQueueViewModule extends AbstractModule {
+    @Provides
+    @Singleton
+    public MailQueueView.Factory provideMailQueueViewFactory() {
+        return new FakeMailQueueView.Factory();
+    }
+}
diff --git a/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/MailQueueViewChoice.java b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/MailQueueViewChoice.java
new file mode 100644
index 0000000000..ad0260f92d
--- /dev/null
+++ b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/MailQueueViewChoice.java
@@ -0,0 +1,67 @@
+/****************************************************************
+ * 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.modules.queue.rabbitmq;
+
+import static org.apache.james.modules.queue.rabbitmq.RabbitMQModule.RABBITMQ_CONFIGURATION_NAME;
+
+import java.io.FileNotFoundException;
+
+import org.apache.commons.configuration2.Configuration;
+import org.apache.commons.configuration2.ex.ConfigurationException;
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.james.utils.PropertiesProvider;
+
+import com.google.inject.Module;
+
+public enum MailQueueViewChoice {
+    CASSANDRA,
+    NONE;
+
+    public static class ModuleChooser {
+        public static Module choose(MailQueueViewChoice choice) {
+            switch (choice) {
+                case CASSANDRA:
+                    return new CassandraMailQueueViewModule();
+                case NONE:
+                    return new FakeMailQueueViewModule();
+                default:
+                    throw new NotImplementedException();
+            }
+        }
+    }
+
+    public static MailQueueViewChoice parse(Configuration configuration) {
+        if (configuration.getBoolean("cassandra.view.enabled", true)) {
+            return CASSANDRA;
+        } else {
+            return NONE;
+        }
+    }
+
+    public static MailQueueViewChoice parse(PropertiesProvider configuration) {
+        try {
+            return parse(configuration.getConfiguration(RABBITMQ_CONFIGURATION_NAME));
+        } catch (FileNotFoundException e) {
+            return CASSANDRA;
+        } catch (ConfigurationException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/RabbitMQModule.java b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/RabbitMQModule.java
index 5c4545da81..712354a21e 100644
--- a/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/RabbitMQModule.java
+++ b/server/container/guice/queue/rabbitmq/src/main/java/org/apache/james/modules/queue/rabbitmq/RabbitMQModule.java
@@ -26,18 +26,12 @@ import javax.inject.Provider;
 import javax.inject.Singleton;
 
 import org.apache.commons.configuration2.ex.ConfigurationException;
-import org.apache.james.backends.cassandra.components.CassandraModule;
 import org.apache.james.backends.rabbitmq.RabbitMQConfiguration;
 import org.apache.james.backends.rabbitmq.RabbitMQHealthCheck;
 import org.apache.james.backends.rabbitmq.ReactorRabbitMQChannelPool;
 import org.apache.james.backends.rabbitmq.ReceiverProvider;
 import org.apache.james.backends.rabbitmq.SimpleConnectionPool;
-import org.apache.james.blob.api.BlobReferenceSource;
 import org.apache.james.core.healthcheck.HealthCheck;
-import org.apache.james.eventsourcing.Event;
-import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTO;
-import org.apache.james.eventsourcing.eventstore.cassandra.dto.EventDTOModule;
-import org.apache.james.lifecycle.api.StartUpCheck;
 import org.apache.james.metrics.api.MetricFactory;
 import org.apache.james.queue.api.MailQueue;
 import org.apache.james.queue.api.MailQueueFactory;
@@ -45,21 +39,6 @@ import org.apache.james.queue.api.ManageableMailQueue;
 import org.apache.james.queue.rabbitmq.RabbitMQMailQueue;
 import org.apache.james.queue.rabbitmq.RabbitMQMailQueueFactory;
 import org.apache.james.queue.rabbitmq.view.RabbitMQMailQueueConfiguration;
-import org.apache.james.queue.rabbitmq.view.api.MailQueueView;
-import org.apache.james.queue.rabbitmq.view.cassandra.BrowseStartDAO;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueBrowser;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueMailDelete;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueMailStore;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueView;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueViewModule;
-import org.apache.james.queue.rabbitmq.view.cassandra.CassandraMailQueueViewStartUpCheck;
-import org.apache.james.queue.rabbitmq.view.cassandra.ContentStartDAO;
-import org.apache.james.queue.rabbitmq.view.cassandra.DeletedMailsDAO;
-import org.apache.james.queue.rabbitmq.view.cassandra.EnqueuedMailsDAO;
-import org.apache.james.queue.rabbitmq.view.cassandra.MailQueueViewBlobReferenceSource;
-import org.apache.james.queue.rabbitmq.view.cassandra.configuration.CassandraMailQueueViewConfiguration;
-import org.apache.james.queue.rabbitmq.view.cassandra.configuration.CassandraMailQueueViewConfigurationModule;
-import org.apache.james.queue.rabbitmq.view.cassandra.configuration.EventsourcingConfigurationManagement;
 import org.apache.james.utils.InitializationOperation;
 import org.apache.james.utils.InitilizationOperationBuilder;
 import org.apache.james.utils.PropertiesProvider;
@@ -69,7 +48,6 @@ import org.slf4j.LoggerFactory;
 import com.google.inject.AbstractModule;
 import com.google.inject.Provides;
 import com.google.inject.Scopes;
-import com.google.inject.TypeLiteral;
 import com.google.inject.multibindings.Multibinder;
 import com.google.inject.multibindings.ProvidesIntoSet;
 
@@ -85,36 +63,12 @@ public class RabbitMQModule extends AbstractModule {
 
     @Override
     protected void configure() {
-        bind(EnqueuedMailsDAO.class).in(Scopes.SINGLETON);
-        bind(DeletedMailsDAO.class).in(Scopes.SINGLETON);
-        bind(BrowseStartDAO.class).in(Scopes.SINGLETON);
-        bind(CassandraMailQueueBrowser.class).in(Scopes.SINGLETON);
-        bind(CassandraMailQueueMailDelete.class).in(Scopes.SINGLETON);
-        bind(CassandraMailQueueMailStore.class).in(Scopes.SINGLETON);
-        bind(ContentStartDAO.class).in(Scopes.SINGLETON);
         bind(SimpleConnectionPool.class).in(Scopes.SINGLETON);
 
-        Multibinder<CassandraModule> cassandraModuleBinder = Multibinder.newSetBinder(binder(), CassandraModule.class);
-        cassandraModuleBinder.addBinding().toInstance(CassandraMailQueueViewModule.MODULE);
-
-        bind(EventsourcingConfigurationManagement.class).in(Scopes.SINGLETON);
-        Multibinder<EventDTOModule<? extends Event, ? extends EventDTO>> eventDTOModuleBinder = Multibinder.newSetBinder(binder(), new TypeLiteral<EventDTOModule<? extends Event, ? extends EventDTO>>() {});
-        eventDTOModuleBinder.addBinding().toInstance(CassandraMailQueueViewConfigurationModule.MAIL_QUEUE_VIEW_CONFIGURATION);
-
-        Multibinder.newSetBinder(binder(), StartUpCheck.class).addBinding().to(CassandraMailQueueViewStartUpCheck.class);
         Multibinder.newSetBinder(binder(), HealthCheck.class).addBinding().to(RabbitMQHealthCheck.class);
 
         Multibinder<SimpleConnectionPool.ReconnectionHandler> reconnectionHandlerMultibinder = Multibinder.newSetBinder(binder(), SimpleConnectionPool.ReconnectionHandler.class);
         reconnectionHandlerMultibinder.addBinding().to(SpoolerReconnectionHandler.class);
-
-        Multibinder.newSetBinder(binder(), BlobReferenceSource.class)
-            .addBinding().to(MailQueueViewBlobReferenceSource.class);
-    }
-
-    @Provides
-    @Singleton
-    public MailQueueView.Factory provideMailQueueViewFactory(CassandraMailQueueView.Factory cassandraMailQueueViewFactory) {
-        return cassandraMailQueueViewFactory;
     }
 
     @Provides
@@ -166,12 +120,6 @@ public class RabbitMQModule extends AbstractModule {
         return RabbitMQConfiguration.from(configuration);
     }
 
-    @Provides
-    @Singleton
-    private CassandraMailQueueViewConfiguration getMailQueueViewConfiguration(@Named(RABBITMQ_CONFIGURATION_NAME) org.apache.commons.configuration2.Configuration configuration) {
-        return CassandraMailQueueViewConfiguration.from(configuration);
-    }
-
     @Provides
     @Singleton
     private RabbitMQMailQueueConfiguration getMailQueueSizeConfiguration(@Named(RABBITMQ_CONFIGURATION_NAME) org.apache.commons.configuration2.Configuration configuration) {
diff --git a/server/queue/queue-rabbitmq/src/main/java/org/apache/james/queue/rabbitmq/view/api/FakeMailQueueView.java b/server/queue/queue-rabbitmq/src/main/java/org/apache/james/queue/rabbitmq/view/api/FakeMailQueueView.java
new file mode 100644
index 0000000000..eecdc5c7bd
--- /dev/null
+++ b/server/queue/queue-rabbitmq/src/main/java/org/apache/james/queue/rabbitmq/view/api/FakeMailQueueView.java
@@ -0,0 +1,80 @@
+/****************************************************************
+ * 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.queue.rabbitmq.view.api;
+
+import java.time.Instant;
+
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.james.queue.api.ManageableMailQueue;
+import org.apache.james.queue.rabbitmq.EnqueueId;
+import org.apache.james.queue.rabbitmq.EnqueuedItem;
+import org.apache.james.queue.rabbitmq.MailQueueName;
+
+import reactor.core.publisher.Flux;
+import reactor.core.publisher.Mono;
+
+public class FakeMailQueueView<V extends ManageableMailQueue.MailQueueItemView> implements MailQueueView<V> {
+    public static class Factory implements MailQueueView.Factory {
+        @Override
+        public MailQueueView create(MailQueueName mailQueueName) {
+            return new FakeMailQueueView();
+        }
+    }
+
+    @Override
+    public void initialize(MailQueueName mailQueueName) {
+
+    }
+
+    @Override
+    public Mono<Void> storeMail(EnqueuedItem enqueuedItem) {
+        return Mono.empty();
+    }
+
+    @Override
+    public Mono<Boolean> isPresent(EnqueueId id) {
+        return Mono.just(true);
+    }
+
+    @Override
+    public long delete(DeleteCondition deleteCondition) {
+        return 0;
+    }
+
+    @Override
+    public ManageableMailQueue.MailQueueIterator browse() {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public Flux<V> browseReactive() {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public Flux<V> browseOlderThanReactive(Instant olderThan) {
+        throw new NotImplementedException();
+    }
+
+    @Override
+    public long getSize() {
+        return 0;
+    }
+}
diff --git a/src/site/xdoc/server/config-rabbitmq.xml b/src/site/xdoc/server/config-rabbitmq.xml
index bbb47fbd9a..9acbc64e22 100644
--- a/src/site/xdoc/server/config-rabbitmq.xml
+++ b/src/site/xdoc/server/config-rabbitmq.xml
@@ -153,6 +153,11 @@
           this implementation relies on Cassandra.
       </p>
       <dl>
+          <dt><strong>cassandra.view.enabled</strong></dt>
+          <dd>Whether the Cassandra administrative view should be activated. Boolean value defaulting to true.
+              Not necessarily needed for MDA deployments, mail queue management adds significant complexity.
+          </dd>
+
           <dt><strong>mailqueue.view.sliceWindow</strong></dt>
           <dd>
               James divide the view into slices, each slice contains data for a given period, sliceWindow parameter controls this period.


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