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 2023/05/02 10:48:35 UTC

[james-project] 01/02: JAMES-3822 RFC-4865 Implement delayed sends in SMTP

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 f2d6902b8b57f55efa9f9f57b3e1ae0c96958613
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Fri Apr 14 08:59:32 2023 +0700

    JAMES-3822 RFC-4865 Implement delayed sends in SMTP
    
    Co-authored-by: Thanh Bui <vt...@linagora.com>
---
 .../protocols/smtp/core/esmtp/EhloCmdHandler.java  |   2 +-
 .../apache/james/protocols/smtp/hook/HeloHook.java |   2 +-
 .../modules/ROOT/pages/configure/smtp-hooks.adoc   |  15 +
 .../apache/james/smtpserver/SendMailHandler.java   |  32 +-
 .../apache/james/smtpserver/dsn/DSNEhloHook.java   |   2 +-
 .../FutureReleaseEHLOHook.java}                    |  31 +-
 .../FutureReleaseMailParameterHook.java            | 123 +++++++
 .../futurerelease/FutureReleaseParameters.java     |  70 ++++
 .../apache/james/smtpserver/FutureReleaseTest.java | 398 +++++++++++++++++++++
 .../FutureReleaseParametersTest.java}              |  25 +-
 .../test/resources/smtpserver-futurerelease.xml    |  50 +++
 11 files changed, 720 insertions(+), 30 deletions(-)

diff --git a/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/core/esmtp/EhloCmdHandler.java b/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/core/esmtp/EhloCmdHandler.java
index 3c81c1cb2b..dc78bdd54c 100644
--- a/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/core/esmtp/EhloCmdHandler.java
+++ b/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/core/esmtp/EhloCmdHandler.java
@@ -162,7 +162,7 @@ public class EhloCmdHandler extends AbstractHookableCmdHandler<HeloHook> impleme
         return ImmutableList.<String>builder()
             .addAll(ESMTP_FEATURES)
             .addAll(getHooks().stream()
-                .flatMap(heloHook -> heloHook.implementedEsmtpFeatures().stream())
+                .flatMap(heloHook -> heloHook.implementedEsmtpFeatures(session).stream())
                 .collect(ImmutableList.toImmutableList()))
             .build();
     }
diff --git a/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/hook/HeloHook.java b/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/hook/HeloHook.java
index 3ca3c59710..49ffb97c2d 100644
--- a/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/hook/HeloHook.java
+++ b/protocols/smtp/src/main/java/org/apache/james/protocols/smtp/hook/HeloHook.java
@@ -32,7 +32,7 @@ public interface HeloHook extends Hook {
     /**
      * @return ESMTP extensions to be advertised as part of EHLO answers
      */
-    default Set<String> implementedEsmtpFeatures() {
+    default Set<String> implementedEsmtpFeatures(SMTPSession session) {
         return ImmutableSet.of();
     }
 
diff --git a/server/apps/distributed-app/docs/modules/ROOT/pages/configure/smtp-hooks.adoc b/server/apps/distributed-app/docs/modules/ROOT/pages/configure/smtp-hooks.adoc
index b7253a4566..ab3a9f2b1b 100644
--- a/server/apps/distributed-app/docs/modules/ROOT/pages/configure/smtp-hooks.adoc
+++ b/server/apps/distributed-app/docs/modules/ROOT/pages/configure/smtp-hooks.adoc
@@ -301,4 +301,19 @@ Example configuration:
     <!-- ... -->
     <handler class="org.apache.james.smtpserver.fastfail.ValidSenderDomainHandler"/>
 </handlerchain>
+....
+
+== FUTURERELEASE hooks
+
+The Distributed server has optional support for FUTURERELEASE (link:https://www.rfc-editor.org/rfc/rfc4865.html[RFC-4865])
+
+....
+<smtpserver enabled="true">
+    <...> <!-- The rest of your SMTP configuration, unchanged -->
+    <handlerchain>
+        <handler class="org.apache.james.smtpserver.futurerelease.FutureReleaseEHLOHook"/>
+        <handler class="org.apache.james.smtpserver.futurerelease.FutureReleaseMailParameterHook"/>
+        <handler class="org.apache.james.smtpserver.CoreCmdHandlerLoader"/>
+    </handlerchain>
+</smtpserver>
 ....
\ No newline at end of file
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/SendMailHandler.java b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/SendMailHandler.java
index a6c155fdd0..4b4296e4e5 100644
--- a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/SendMailHandler.java
+++ b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/SendMailHandler.java
@@ -19,23 +19,30 @@
 
 package org.apache.james.smtpserver;
 
+import static org.apache.james.smtpserver.futurerelease.FutureReleaseMailParameterHook.FUTURERELEASE_HOLDFOR;
+
 import java.io.Closeable;
 import java.io.IOException;
+import java.util.Optional;
 
 import javax.inject.Inject;
 
 import org.apache.commons.configuration2.Configuration;
+import org.apache.james.protocols.api.ProtocolSession;
 import org.apache.james.protocols.smtp.SMTPSession;
 import org.apache.james.protocols.smtp.dsn.DSNStatus;
 import org.apache.james.protocols.smtp.hook.HookResult;
 import org.apache.james.protocols.smtp.hook.HookReturnCode;
 import org.apache.james.queue.api.MailQueue;
 import org.apache.james.queue.api.MailQueueFactory;
+import org.apache.james.smtpserver.futurerelease.FutureReleaseParameters;
 import org.apache.james.util.MDCBuilder;
 import org.apache.mailet.Mail;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.fge.lambdas.Throwing;
+
 /**
  * Queue the message
  */
@@ -72,12 +79,25 @@ public class SendMailHandler implements JamesMessageHook {
         LOGGER.debug("sending mail");
 
         try (Closeable closeable = MDCBuilder.ofValue("messageId", mail.getMessage().getMessageID()).build()) {
-            queue.enQueue(mail);
-            LOGGER.info("Successfully spooled mail {} with messageId {} from {} on {} for {}", mail.getName(),
-                mail.getMessage().getMessageID(),
-                mail.getMaybeSender().asString(),
-                session.getRemoteAddress().getAddress(),
-                mail.getRecipients());
+            Optional<FutureReleaseParameters.HoldFor> delays = session.getAttachment(FUTURERELEASE_HOLDFOR, ProtocolSession.State.Transaction);
+
+            delays.ifPresentOrElse(Throwing.consumer(holdFor -> {
+                    queue.enQueue(mail, holdFor.value());
+                    LOGGER.info("Successfully spooled mail {} with messageId {} from {} on {} for {} with delay {}", mail.getName(),
+                        mail.getMessage().getMessageID(),
+                        mail.getMaybeSender().asString(),
+                        session.getRemoteAddress().getAddress(),
+                        mail.getRecipients(),
+                        holdFor.value());
+                }),
+                Throwing.runnable(() -> {
+                    queue.enQueue(mail);
+                    LOGGER.info("Successfully spooled mail {} with messageId {} from {} on {} for {}", mail.getName(),
+                        mail.getMessage().getMessageID(),
+                        mail.getMaybeSender().asString(),
+                        session.getRemoteAddress().getAddress(),
+                        mail.getRecipients());
+                }));
         } catch (Exception me) {
             LOGGER.error("Unknown error occurred while processing DATA.", me);
             return HookResult.builder()
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
index a10a340fb3..5fe25e3dd4 100644
--- a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
+++ b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
@@ -29,7 +29,7 @@ import com.google.common.collect.ImmutableSet;
 
 public class DSNEhloHook implements HeloHook {
     @Override
-    public Set<String> implementedEsmtpFeatures() {
+    public Set<String> implementedEsmtpFeatures(SMTPSession session) {
         return ImmutableSet.of("DSN");
     }
 
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseEHLOHook.java
similarity index 60%
copy from server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
copy to server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseEHLOHook.java
index a10a340fb3..767afad1e7 100644
--- a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
+++ b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseEHLOHook.java
@@ -16,21 +16,43 @@
  * specific language governing permissions and limitations      *
  * under the License.                                           *
  ****************************************************************/
+package org.apache.james.smtpserver.futurerelease;
 
-package org.apache.james.smtpserver.dsn;
+import static org.apache.james.smtpserver.futurerelease.FutureReleaseParameters.MAX_HOLD_FOR_SUPPORTED;
 
+import java.time.Clock;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
 import java.util.Set;
 
+import javax.inject.Inject;
+
 import org.apache.james.protocols.smtp.SMTPSession;
 import org.apache.james.protocols.smtp.hook.HeloHook;
 import org.apache.james.protocols.smtp.hook.HookResult;
 
 import com.google.common.collect.ImmutableSet;
 
-public class DSNEhloHook implements HeloHook {
+public class FutureReleaseEHLOHook implements HeloHook {
+    private final Clock clock;
+
+    @Inject
+    public FutureReleaseEHLOHook(Clock clock) {
+        this.clock = clock;
+    }
+
     @Override
-    public Set<String> implementedEsmtpFeatures() {
-        return ImmutableSet.of("DSN");
+    public Set<String> implementedEsmtpFeatures(SMTPSession session) {
+        if (session.getUsername() != null) {
+            Instant now = LocalDateTime.now(clock).toInstant(ZoneOffset.UTC);
+            String dateAsString = DateTimeFormatter.ISO_INSTANT.withZone(ZoneId.of("UTC")).format(now.plus(MAX_HOLD_FOR_SUPPORTED));
+
+            return ImmutableSet.of("FUTURERELEASE " + MAX_HOLD_FOR_SUPPORTED.toSeconds() + " " + dateAsString);
+        }
+        return ImmutableSet.of();
     }
 
     @Override
@@ -38,3 +60,4 @@ public class DSNEhloHook implements HeloHook {
         return HookResult.DECLINED;
     }
 }
+
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseMailParameterHook.java b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseMailParameterHook.java
new file mode 100644
index 0000000000..a215f6a72d
--- /dev/null
+++ b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseMailParameterHook.java
@@ -0,0 +1,123 @@
+/****************************************************************
+ * 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.smtpserver.futurerelease;
+
+import static org.apache.james.protocols.api.ProtocolSession.State.Transaction;
+import static org.apache.james.smtpserver.futurerelease.FutureReleaseParameters.HOLDFOR_PARAMETER;
+import static org.apache.james.smtpserver.futurerelease.FutureReleaseParameters.HOLDUNTIL_PARAMETER;
+import static org.apache.james.smtpserver.futurerelease.FutureReleaseParameters.MAX_HOLD_FOR_SUPPORTED;
+
+import java.time.Clock;
+import java.time.Duration;
+import java.time.Instant;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.time.format.DateTimeFormatter;
+
+import javax.inject.Inject;
+
+import org.apache.james.protocols.api.ProtocolSession;
+import org.apache.james.protocols.smtp.SMTPRetCode;
+import org.apache.james.protocols.smtp.SMTPSession;
+import org.apache.james.protocols.smtp.hook.HookResult;
+import org.apache.james.protocols.smtp.hook.HookReturnCode;
+import org.apache.james.protocols.smtp.hook.MailParametersHook;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FutureReleaseMailParameterHook implements MailParametersHook {
+    private static final Logger LOGGER = LoggerFactory.getLogger(FutureReleaseMailParameterHook.class);
+
+    public static final ProtocolSession.AttachmentKey<FutureReleaseParameters.HoldFor> FUTURERELEASE_HOLDFOR = ProtocolSession.AttachmentKey.of("FUTURERELEASE_HOLDFOR", FutureReleaseParameters.HoldFor.class);
+
+    private final Clock clock;
+
+    @Inject
+    public FutureReleaseMailParameterHook(Clock clock) {
+        this.clock = clock;
+    }
+
+    @Override
+    public HookResult doMailParameter(SMTPSession session, String paramName, String paramValue) {
+        if (session.getUsername() == null) {
+            LOGGER.debug("Needs to be logged in in order to use future release extension");
+            return HookResult.builder()
+                .hookReturnCode(HookReturnCode.deny())
+                .smtpDescription("Needs to be logged in in order to use future release extension")
+                .build();
+        }
+
+        try {
+            Duration requestedHoldFor = evaluateHoldFor(paramName, paramValue);
+
+            if (requestedHoldFor.compareTo(MAX_HOLD_FOR_SUPPORTED) > 0) {
+                LOGGER.debug("HoldFor is greater than max-future-release-interval or holdUntil exceeded max-future-release-date-time");
+                return HookResult.builder()
+                    .smtpReturnCode(SMTPRetCode.SYNTAX_ERROR_ARGUMENTS)
+                    .hookReturnCode(HookReturnCode.deny())
+                    .smtpDescription("HoldFor is greater than max-future-release-interval or holdUntil exceeded max-future-release-date-time")
+                    .build();
+            }
+            if (requestedHoldFor.isNegative()) {
+                LOGGER.debug("HoldFor value is negative or holdUntil value is before now");
+                return HookResult.builder()
+                    .hookReturnCode(HookReturnCode.deny())
+                    .smtpReturnCode(SMTPRetCode.SYNTAX_ERROR_ARGUMENTS)
+                    .smtpDescription("HoldFor value is negative or holdUntil value is before now")
+                    .build();
+            }
+            if (session.getAttachment(FUTURERELEASE_HOLDFOR, Transaction).isPresent()) {
+                LOGGER.debug("Mail parameter cannot contains both holdFor and holdUntil parameters");
+                return HookResult.builder()
+                    .hookReturnCode(HookReturnCode.deny())
+                    .smtpDescription("Mail parameter cannot contains both holdFor and holdUntil parameters")
+                    .build();
+            }
+            session.setAttachment(FUTURERELEASE_HOLDFOR, FutureReleaseParameters.HoldFor.of(requestedHoldFor), Transaction);
+            return HookResult.DECLINED;
+        } catch (IllegalArgumentException e) {
+            LOGGER.debug("Incorrect syntax when handling FUTURE-RELEASE mail parameter", e);
+            return HookResult.builder()
+                .hookReturnCode(HookReturnCode.deny())
+                .smtpReturnCode(SMTPRetCode.SYNTAX_ERROR_ARGUMENTS)
+                .smtpDescription("Incorrect syntax when handling FUTURE-RELEASE mail parameter")
+                .build();
+        }
+    }
+
+    private Duration evaluateHoldFor(String paramName, String paramValue) {
+        if (paramName.equals(HOLDFOR_PARAMETER)) {
+            return Duration.ofSeconds(Long.parseLong(paramValue));
+        }
+        if (paramName.equals(HOLDUNTIL_PARAMETER)) {
+            DateTimeFormatter formatter = DateTimeFormatter.ISO_INSTANT.withZone(ZoneId.of("Z"));
+            Instant now = LocalDateTime.now(clock).toInstant(ZoneOffset.UTC);
+            return Duration.between(now, ZonedDateTime.parse(paramValue, formatter).toInstant());
+        }
+        throw new IllegalArgumentException("Invalid parameter name " + paramName);
+    }
+
+    @Override
+    public String[] getMailParamNames() {
+        return new String[] {HOLDFOR_PARAMETER, HOLDUNTIL_PARAMETER};
+    }
+}
\ No newline at end of file
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParameters.java b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParameters.java
new file mode 100644
index 0000000000..cb297ed5b9
--- /dev/null
+++ b/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParameters.java
@@ -0,0 +1,70 @@
+/****************************************************************
+ * 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.smtpserver.futurerelease;
+
+import java.time.Duration;
+import java.util.Objects;
+
+import com.google.common.base.MoreObjects;
+import com.google.common.base.Preconditions;
+
+public class FutureReleaseParameters {
+    public static final String HOLDFOR_PARAMETER = "HOLDFOR";
+    public static final String HOLDUNTIL_PARAMETER = "HOLDUNTIL";
+    public static final Duration MAX_HOLD_FOR_SUPPORTED = Duration.ofDays(1);
+
+    public static class HoldFor {
+        public static HoldFor of(Duration value) {
+            Preconditions.checkNotNull(value);
+            return new HoldFor(value);
+        }
+
+        private final Duration value;
+
+        private HoldFor(Duration value) {
+            this.value = value;
+        }
+
+        public Duration value() {
+            return value;
+        }
+
+        @Override
+        public final boolean equals(Object o) {
+            if (o instanceof HoldFor) {
+                HoldFor holdFor = (HoldFor) o;
+                return Objects.equals(this.value, holdFor.value);
+            }
+            return false;
+        }
+
+        @Override
+        public final int hashCode() {
+            return Objects.hash(value);
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(this)
+                .add("value", value)
+                .toString();
+        }
+    }
+}
diff --git a/server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/FutureReleaseTest.java b/server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/FutureReleaseTest.java
new file mode 100644
index 0000000000..dc87df8f8e
--- /dev/null
+++ b/server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/FutureReleaseTest.java
@@ -0,0 +1,398 @@
+/****************************************************************
+ * 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.smtpserver;
+
+import com.google.common.collect.ImmutableList;
+import com.google.inject.TypeLiteral;
+import org.apache.commons.configuration2.BaseHierarchicalConfiguration;
+import org.apache.commons.net.smtp.SMTPClient;
+import org.apache.james.UserEntityValidator;
+import org.apache.james.core.Domain;
+import org.apache.james.core.Username;
+import org.apache.james.dnsservice.api.DNSService;
+import org.apache.james.dnsservice.api.InMemoryDNSService;
+import org.apache.james.domainlist.api.DomainList;
+import org.apache.james.domainlist.lib.DomainListConfiguration;
+import org.apache.james.domainlist.memory.MemoryDomainList;
+import org.apache.james.filesystem.api.FileSystem;
+import org.apache.james.mailbox.Authorizator;
+import org.apache.james.mailrepository.api.MailRepositoryStore;
+import org.apache.james.mailrepository.api.Protocol;
+import org.apache.james.mailrepository.memory.*;
+import org.apache.james.metrics.api.Metric;
+import org.apache.james.metrics.api.MetricFactory;
+import org.apache.james.metrics.tests.RecordingMetricFactory;
+import org.apache.james.protocols.api.utils.ProtocolServerUtils;
+import org.apache.james.protocols.lib.mock.MockProtocolHandlerLoader;
+import org.apache.james.queue.api.MailQueueFactory;
+import org.apache.james.queue.api.ManageableMailQueue;
+import org.apache.james.queue.api.RawMailQueueItemDecoratorFactory;
+import org.apache.james.queue.memory.MemoryMailQueueFactory;
+import org.apache.james.rrt.api.AliasReverseResolver;
+import org.apache.james.rrt.api.CanSendFrom;
+import org.apache.james.rrt.api.RecipientRewriteTable;
+import org.apache.james.rrt.api.RecipientRewriteTableConfiguration;
+import org.apache.james.rrt.lib.AliasReverseResolverImpl;
+import org.apache.james.rrt.lib.CanSendFromImpl;
+import org.apache.james.rrt.memory.MemoryRecipientRewriteTable;
+import org.apache.james.server.core.configuration.Configuration;
+import org.apache.james.server.core.configuration.FileConfigurationProvider;
+import org.apache.james.server.core.filesystem.FileSystemImpl;
+import org.apache.james.smtpserver.netty.SMTPServer;
+import org.apache.james.smtpserver.netty.SmtpMetricsImpl;
+import org.apache.james.user.api.UsersRepository;
+import org.apache.james.user.memory.MemoryUsersRepository;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.time.Clock;
+import java.time.Instant;
+import java.time.ZoneId;
+import java.time.chrono.ChronoZonedDateTime;
+import java.util.Base64;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+class FutureReleaseTest {
+    public static final String LOCAL_DOMAIN = "example.local";
+    public static final Username BOB = Username.of("bob@localhost");
+    public static final String PASSWORD = "bobpwd";
+    private static final Instant DATE = Instant.parse("2023-04-14T10:00:00.00Z");
+    private static final Clock CLOCK = Clock.fixed(DATE, ZoneId.of("Z"));
+
+    protected MemoryDomainList domainList;
+    protected MemoryUsersRepository usersRepository;
+    protected SMTPServerTest.AlterableDNSServer dnsServer;
+    protected MemoryMailRepositoryStore mailRepositoryStore;
+    protected FileSystemImpl fileSystem;
+    protected Configuration configuration;
+    protected MockProtocolHandlerLoader chain;
+    protected MemoryMailQueueFactory queueFactory;
+    protected MemoryMailQueueFactory.MemoryCacheableMailQueue queue;
+
+    private SMTPServer smtpServer;
+
+    @BeforeEach
+    void setUp() throws Exception {
+        domainList = new MemoryDomainList(new InMemoryDNSService());
+        domainList.configure(DomainListConfiguration.DEFAULT);
+
+        domainList.addDomain(Domain.of(LOCAL_DOMAIN));
+        domainList.addDomain(Domain.of("examplebis.local"));
+        usersRepository = MemoryUsersRepository.withVirtualHosting(domainList);
+        usersRepository.addUser(BOB, PASSWORD);
+
+        createMailRepositoryStore();
+
+        setUpFakeLoader();
+        setUpSMTPServer();
+
+        smtpServer.configure(FileConfigurationProvider.getConfig(
+            ClassLoader.getSystemResourceAsStream("smtpserver-futurerelease.xml")));
+        smtpServer.init();
+    }
+
+    protected void createMailRepositoryStore() throws Exception {
+        configuration = Configuration.builder()
+            .workingDirectory("../")
+            .configurationFromClasspath()
+            .build();
+        fileSystem = new FileSystemImpl(configuration.directories());
+        MemoryMailRepositoryUrlStore urlStore = new MemoryMailRepositoryUrlStore();
+
+        MailRepositoryStoreConfiguration configuration = MailRepositoryStoreConfiguration.forItems(
+            new MailRepositoryStoreConfiguration.Item(
+                ImmutableList.of(new Protocol("memory")),
+                MemoryMailRepository.class.getName(),
+                new BaseHierarchicalConfiguration()));
+
+        mailRepositoryStore = new MemoryMailRepositoryStore(urlStore, new SimpleMailRepositoryLoader(), configuration);
+        mailRepositoryStore.init();
+    }
+
+    protected SMTPServer createSMTPServer(SmtpMetricsImpl smtpMetrics) {
+        return new SMTPServer(smtpMetrics);
+    }
+
+    protected void setUpSMTPServer() {
+        SmtpMetricsImpl smtpMetrics = mock(SmtpMetricsImpl.class);
+        when(smtpMetrics.getCommandsMetric()).thenReturn(mock(Metric.class));
+        when(smtpMetrics.getConnectionMetric()).thenReturn(mock(Metric.class));
+        smtpServer = createSMTPServer(smtpMetrics);
+        smtpServer.setDnsService(dnsServer);
+        smtpServer.setFileSystem(fileSystem);
+        smtpServer.setProtocolHandlerLoader(chain);
+    }
+
+    protected void setUpFakeLoader() {
+        dnsServer = new SMTPServerTest.AlterableDNSServer();
+
+        MemoryRecipientRewriteTable rewriteTable = new MemoryRecipientRewriteTable();
+        rewriteTable.setConfiguration(RecipientRewriteTableConfiguration.DEFAULT_ENABLED);
+        AliasReverseResolver aliasReverseResolver = new AliasReverseResolverImpl(rewriteTable);
+        CanSendFrom canSendFrom = new CanSendFromImpl(rewriteTable, aliasReverseResolver);
+        queueFactory = new MemoryMailQueueFactory(new RawMailQueueItemDecoratorFactory(), CLOCK);
+        queue = queueFactory.createQueue(MailQueueFactory.SPOOL);
+
+        chain = MockProtocolHandlerLoader.builder()
+            .put(binder -> binder.bind(DomainList.class).toInstance(domainList))
+            .put(binder -> binder.bind(Clock.class).toInstance(CLOCK))
+            .put(binder -> binder.bind(new TypeLiteral<MailQueueFactory<?>>() {}).toInstance(queueFactory))
+            .put(binder -> binder.bind(RecipientRewriteTable.class).toInstance(rewriteTable))
+            .put(binder -> binder.bind(CanSendFrom.class).toInstance(canSendFrom))
+            .put(binder -> binder.bind(FileSystem.class).toInstance(fileSystem))
+            .put(binder -> binder.bind(MailRepositoryStore.class).toInstance(mailRepositoryStore))
+            .put(binder -> binder.bind(DNSService.class).toInstance(dnsServer))
+            .put(binder -> binder.bind(UsersRepository.class).toInstance(usersRepository))
+            .put(binder -> binder.bind(MetricFactory.class).to(RecordingMetricFactory.class))
+            .put(binder -> binder.bind(UserEntityValidator.class).toInstance(UserEntityValidator.NOOP))
+            .put(binder -> binder.bind(Authorizator.class).toInstance((userId, otherUserId) -> Authorizator.AuthorizationState.ALLOWED))
+            .build();
+    }
+
+    @AfterEach
+    void tearDown() {
+        smtpServer.destroy();
+    }
+
+    @Test
+    void rejectFutureReleaseUsageWhenUnauthenticated() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+
+        smtpProtocol.sendCommand("EHLO whatever.tld");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@whatever.tld> HOLDFOR=83200");
+
+        assertThat(smtpProtocol.getReplyString()).isEqualTo("554 Needs to be logged in in order to use future release extension\r\n");
+    }
+
+    @Test
+    void ehloShouldAdvertiseFutureReleaseExtension() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+        smtpProtocol.sendCommand("EHLO localhost");
+
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(250);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("250 FUTURERELEASE 86400 2023-04-15T10:00:00Z");
+        });
+    }
+
+    @Test
+    void ehloShouldNotAdvertiseFutureReleaseExtensionWhenUnauthenticated() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        smtpProtocol.sendCommand("EHLO localhost");
+
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(250);
+            softly.assertThat(smtpProtocol.getReplyString()).doesNotContain("250 FUTURERELEASE 86400 2023-04-15T10:00:00Z");
+        });
+    }
+
+    private void authenticate(SMTPClient smtpProtocol) throws IOException {
+        smtpProtocol.sendCommand("AUTH PLAIN");
+        smtpProtocol.sendCommand(Base64.getEncoder().encodeToString(("\0" + BOB.asString() + "\0" + PASSWORD + "\0").getBytes(UTF_8)));
+        assertThat(smtpProtocol.getReplyCode())
+            .as("authenticated")
+            .isEqualTo(235);
+    }
+
+    @Test
+    void testSuccessCaseWithHoldForParams() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDFOR=83200");
+        smtpProtocol.sendCommand("RCPT TO:<rc...@localhost>");
+        smtpProtocol.sendShortMessageData("Subject: test mail\r\n\r\nTest body testSimpleMailSendWithFutureRelease\r\n.\r\n");
+
+        ManageableMailQueue.MailQueueIterator browse = queue.browse();
+        assertThat(browse.hasNext()).isTrue();
+        assertThat(browse.next().getNextDelivery().map(ChronoZonedDateTime::toInstant))
+            .contains(DATE.plusSeconds(83200));
+    }
+
+    @Test
+    void testSuccessCaseWithHoldUntilParams() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDUNTIL=2023-04-14T10:30:00Z");
+        smtpProtocol.sendCommand("RCPT TO:<rc...@localhost>");
+        smtpProtocol.sendShortMessageData("Subject: test mail\r\n\r\nTest body testSimpleMailSendWithFutureRelease\r\n.\r\n");
+
+        ManageableMailQueue.MailQueueIterator browse = queue.browse();
+        assertThat(browse.hasNext()).isTrue();
+        assertThat(browse.next().getNextDelivery().map(ChronoZonedDateTime::toInstant))
+            .contains(Instant.parse("2023-04-14T10:30:00Z"));
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenExceedingMaxFutureReleaseInterval() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDFOR=93200");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 HoldFor is greater than max-future-release-interval or holdUntil exceeded max-future-release-date-time");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenInvalidHoldFor() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDFOR=BAD");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 Incorrect syntax when handling FUTURE-RELEASE mail parameter");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenInvalidHoldUntil() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDUNTIL=BAD");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isNotEqualTo(250);
+            softly.assertThat(smtpProtocol.getReplyString()).doesNotContain("250");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenHoldUntilIsADate() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDUNTIL=2023-04-15");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isNotEqualTo(250);
+            softly.assertThat(smtpProtocol.getReplyString()).doesNotContain("250");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenMaxFutureReleaseDateTime() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDUNTIL=2023-04-15T11:00:00Z");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 HoldFor is greater than max-future-release-interval or holdUntil exceeded max-future-release-date-time");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenHoldForIsNegative() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDFOR=-30");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 HoldFor value is negative or holdUntil value is before now");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenHoldUntilBeforeNow() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDUNTIL=2023-04-13T05:00:00Z");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 HoldFor value is negative or holdUntil value is before now");
+        });
+    }
+
+    @Test
+    void mailShouldBeRejectedWhenMailParametersContainBothHoldForAndHoldUntil() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost> HOLDFOR=83017 HOLDUNTIL=2023-04-12T11:00:00Z");
+        SoftAssertions.assertSoftly(softly -> {
+            softly.assertThat(smtpProtocol.getReplyCode()).isEqualTo(501);
+            softly.assertThat(smtpProtocol.getReplyString()).contains("501 HoldFor value is negative or holdUntil value is before now");
+        });
+    }
+
+    @Test
+    void mailShouldBeSentWhenThereIsNoMailParameters() throws Exception {
+        SMTPClient smtpProtocol = new SMTPClient();
+        InetSocketAddress bindedAddress = new ProtocolServerUtils(smtpServer).retrieveBindedAddress();
+        smtpProtocol.connect(bindedAddress.getAddress().getHostAddress(), bindedAddress.getPort());
+        authenticate(smtpProtocol);
+
+        smtpProtocol.sendCommand("EHLO localhost");
+        smtpProtocol.sendCommand("MAIL FROM: <bo...@localhost>");
+        smtpProtocol.sendCommand("RCPT TO:<rc...@localhost>");
+        smtpProtocol.sendShortMessageData("Subject: test mail\r\n\r\nTest body testSimpleMailSendWithFutureRelease\r\n.\r\n");
+
+        assertThat(queue.getSize()).isEqualTo(1L);
+    }
+}
diff --git a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java b/server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParametersTest.java
similarity index 68%
copy from server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
copy to server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParametersTest.java
index a10a340fb3..650d916f38 100644
--- a/server/protocols/protocols-smtp/src/main/java/org/apache/james/smtpserver/dsn/DSNEhloHook.java
+++ b/server/protocols/protocols-smtp/src/test/java/org/apache/james/smtpserver/futurerelease/FutureReleaseParametersTest.java
@@ -17,24 +17,15 @@
  * under the License.                                           *
  ****************************************************************/
 
-package org.apache.james.smtpserver.dsn;
+package org.apache.james.smtpserver.futurerelease;
 
-import java.util.Set;
+import org.junit.jupiter.api.Test;
 
-import org.apache.james.protocols.smtp.SMTPSession;
-import org.apache.james.protocols.smtp.hook.HeloHook;
-import org.apache.james.protocols.smtp.hook.HookResult;
+import nl.jqno.equalsverifier.EqualsVerifier;
 
-import com.google.common.collect.ImmutableSet;
-
-public class DSNEhloHook implements HeloHook {
-    @Override
-    public Set<String> implementedEsmtpFeatures() {
-        return ImmutableSet.of("DSN");
-    }
-
-    @Override
-    public HookResult doHelo(SMTPSession session, String helo) {
-        return HookResult.DECLINED;
+class FutureReleaseParametersTest {
+    @Test
+    void testEqualsVerifiersForHoldForClass() {
+        EqualsVerifier.forClass(FutureReleaseParameters.HoldFor.class).verify();
     }
-}
+}
\ No newline at end of file
diff --git a/server/protocols/protocols-smtp/src/test/resources/smtpserver-futurerelease.xml b/server/protocols/protocols-smtp/src/test/resources/smtpserver-futurerelease.xml
new file mode 100644
index 0000000000..d5b4327cf1
--- /dev/null
+++ b/server/protocols/protocols-smtp/src/test/resources/smtpserver-futurerelease.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0"?>
+
+<!--
+  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.
+ -->
+
+<!-- Read https://james.apache.org/server/config-smtp-lmtp.html#SMTP_Configuration for further details -->
+
+<smtpserver enabled="true">
+    <bind>0.0.0.0:0</bind>
+    <connectionBacklog>200</connectionBacklog>
+    <tls socketTLS="false" startTLS="false">
+        <keystore>file://conf/keystore</keystore>
+        <secret>james72laBalle</secret>
+        <provider>org.bouncycastle.jce.provider.BouncyCastleProvider</provider>
+        <algorithm>SunX509</algorithm>
+    </tls>
+    <connectiontimeout>360</connectiontimeout>
+    <connectionLimit>0</connectionLimit>
+    <connectionLimitPerIP>0</connectionLimitPerIP>
+    <auth>
+        <announce>forUnauthorizedAddresses</announce>
+        <requireSSL>false</requireSSL>
+    </auth>
+    <verifyIdentity>true</verifyIdentity>
+    <maxmessagesize>0</maxmessagesize>
+    <addressBracketsEnforcement>true</addressBracketsEnforcement>
+    <smtpGreeting>Apache JAMES awesome SMTP Server</smtpGreeting>
+    <handlerchain>
+        <handler class="org.apache.james.smtpserver.futurerelease.FutureReleaseEHLOHook"/>
+        <handler class="org.apache.james.smtpserver.futurerelease.FutureReleaseMailParameterHook"/>
+        <handler class="org.apache.james.smtpserver.CoreCmdHandlerLoader"/>
+    </handlerchain>
+    <gracefulShutdown>false</gracefulShutdown>
+</smtpserver>
\ No newline at end of file


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