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

[james-project] branch master updated (5331338 -> e461a3a)

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

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


    from 5331338  Fix anchor links
     new 3c78672  JAMES-3295 handle configuration AtMostMatcher with differents config
     new c312298  JAMES-3295 Matcher for temporary delivery failure
     new 9711f90  JAMES-3295 Temporary delivery error matcher implementation/test
     new 5ef6acc  JAMES-3295 add a test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery in RemoteDeliveryErrorHandlingTest
     new e461a3a  JAMES-3295 proof for the test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery

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


Summary of changes:
 .../apache/james/transport/matchers/AtMost.java    |  57 +++++-
 .../james/transport/matchers/AtMostTest.java       | 221 ++++++++++++++-------
 .../mailets/RemoteDeliveryErrorHandlingTest.java   |  29 ++-
 .../mailets/remote/delivery/DeliveryRunnable.java  |   1 +
 ...or.java => IsRemoteDeliveryTemporaryError.java} |  10 +-
 .../remote/delivery/DeliveryRunnableTest.java      |   7 +
 ...ava => IsRemoteDeliveryTemporaryErrorTest.java} |  19 +-
 7 files changed, 246 insertions(+), 98 deletions(-)
 copy server/mailet/mailets/src/main/java/org/apache/james/transport/matchers/{IsRemoteDeliveryPermanentError.java => IsRemoteDeliveryTemporaryError.java} (85%)
 copy server/mailet/mailets/src/test/java/org/apache/james/transport/matchers/{IsRemoteDeliveryPermanentErrorTest.java => IsRemoteDeliveryTemporaryErrorTest.java} (90%)


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


[james-project] 05/05: JAMES-3295 proof for the test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery

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

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

commit e461a3acc5cc7822375019b0efd5b2eabaec71ff
Author: Rene Cordier <rc...@linagora.com>
AuthorDate: Thu Jul 9 11:54:50 2020 +0700

    JAMES-3295 proof for the test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery
---
 .../mailets/RemoteDeliveryErrorHandlingTest.java          | 15 +++++++++++----
 1 file changed, 11 insertions(+), 4 deletions(-)

diff --git a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
index 690e359..a818426 100644
--- a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
+++ b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
@@ -49,6 +49,7 @@ import org.apache.james.transport.matchers.All;
 import org.apache.james.transport.matchers.AtMost;
 import org.apache.james.transport.matchers.IsRemoteDeliveryPermanentError;
 import org.apache.james.transport.matchers.IsRemoteDeliveryTemporaryError;
+import org.apache.james.transport.matchers.RecipientIs;
 import org.apache.james.utils.DataProbeImpl;
 import org.apache.james.utils.MailRepositoryProbeImpl;
 import org.apache.james.utils.SMTPMessageSender;
@@ -57,7 +58,6 @@ import org.apache.james.utils.WebAdminGuiceProbe;
 import org.apache.james.webadmin.WebAdminUtils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.RegisterExtension;
 import org.junit.jupiter.api.io.TempDir;
@@ -101,6 +101,11 @@ public class RemoteDeliveryErrorHandlingTest {
                 .putProcessor(ProcessorConfiguration.transport()
                     .addMailet(BCC_STRIPPER)
                     .addMailet(MailetConfiguration.builder()
+                        .mailet(ToProcessor.class)
+                        .addProperty("processor", "remote-delivery-error")
+                        .matcher(RecipientIs.class)
+                        .matcherCondition(RECIPIENT2))
+                    .addMailet(MailetConfiguration.builder()
                         .mailet(RemoteDelivery.class)
                         .addProperty("maxRetries", "1")
                         .addProperty("delayTime", "0")
@@ -120,7 +125,11 @@ public class RemoteDeliveryErrorHandlingTest {
                     .addMailet(MailetConfiguration.builder()
                         .matcher(IsRemoteDeliveryTemporaryError.class)
                         .mailet(ToRepository.class)
-                        .addProperty("repositoryPath", REMOTE_DELIVERY_TEMPORARY_ERROR_REPOSITORY.asString()))))
+                        .addProperty("repositoryPath", REMOTE_DELIVERY_TEMPORARY_ERROR_REPOSITORY.asString()))
+                    .addMailet(MailetConfiguration.builder()
+                        .matcher(All.class)
+                        .mailet(ToRepository.class)
+                        .addProperty("repositoryPath", ERROR_REPOSITORY.asString()))))
             .build(tempDir);
 
         jamesServer.start();
@@ -262,8 +271,6 @@ public class RemoteDeliveryErrorHandlingTest {
     }
 
     @Test
-    @Disabled("JAMES-3295 we need to add some more mailets to prove that if a mail has no temporary and permanent"
-        + "failure, it can be treated differently")
     void remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery(SMTPMessageSender smtpMessageSender) throws Exception {
         // When we relay a mail where some unexpected accident happens
         smtpMessageSender.connect(LOCALHOST, jamesServer.getProbe(SmtpGuiceProbe.class).getSmtpPort())


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


[james-project] 03/05: JAMES-3295 Temporary delivery error matcher implementation/test

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

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

commit 9711f9046cce9c4e2486d35e64109d5961ecc962
Author: Rene Cordier <rc...@linagora.com>
AuthorDate: Thu Jul 9 10:49:25 2020 +0700

    JAMES-3295 Temporary delivery error matcher implementation/test
---
 .../james/transport/mailets/RemoteDeliveryErrorHandlingTest.java   | 3 ++-
 .../james/transport/mailets/remote/delivery/DeliveryRunnable.java  | 1 +
 .../transport/mailets/remote/delivery/DeliveryRunnableTest.java    | 7 +++++++
 3 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
index 991d0f7..d7519e7 100644
--- a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
+++ b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
@@ -48,6 +48,7 @@ import org.apache.james.queue.api.MailQueueFactory;
 import org.apache.james.transport.matchers.All;
 import org.apache.james.transport.matchers.AtMost;
 import org.apache.james.transport.matchers.IsRemoteDeliveryPermanentError;
+import org.apache.james.transport.matchers.IsRemoteDeliveryTemporaryError;
 import org.apache.james.utils.DataProbeImpl;
 import org.apache.james.utils.MailRepositoryProbeImpl;
 import org.apache.james.utils.SMTPMessageSender;
@@ -114,7 +115,7 @@ public class RemoteDeliveryErrorHandlingTest {
                         .matcher(IsRemoteDeliveryPermanentError.class)
                         .addProperty("repositoryPath", REMOTE_DELIVERY_PERMANENT_ERROR_REPOSITORY.asString()))
                     .addMailet(MailetConfiguration.builder()
-                        .matcher(All.class)
+                        .matcher(IsRemoteDeliveryTemporaryError.class)
                         .mailet(ToRepository.class)
                         .addProperty("repositoryPath", REMOTE_DELIVERY_TEMPORARY_ERROR_REPOSITORY.asString()))))
             .build(tempDir);
diff --git a/server/mailet/mailets/src/main/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnable.java b/server/mailet/mailets/src/main/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnable.java
index 36e65e2..d16d502 100644
--- a/server/mailet/mailets/src/main/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnable.java
+++ b/server/mailet/mailets/src/main/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnable.java
@@ -150,6 +150,7 @@ public class DeliveryRunnable implements Disposable {
             DeliveryRetriesHelper.initRetries(mail);
             mail.setLastUpdated(dateSupplier.get());
         }
+        mail.setAttribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)));
         int retries = DeliveryRetriesHelper.retrieveRetries(mail);
 
         if (retries < configuration.getMaxRetries()) {
diff --git a/server/mailet/mailets/src/test/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnableTest.java b/server/mailet/mailets/src/test/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnableTest.java
index ece78d0..671ce3a 100644
--- a/server/mailet/mailets/src/test/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnableTest.java
+++ b/server/mailet/mailets/src/test/java/org/apache/james/transport/mailets/remote/delivery/DeliveryRunnableTest.java
@@ -19,6 +19,7 @@
 
 package org.apache.james.transport.mailets.remote.delivery;
 
+import static org.apache.james.transport.mailets.remote.delivery.Bouncer.IS_DELIVERY_PERMANENT_ERROR;
 import static org.apache.james.transport.mailets.remote.delivery.DeliveryRunnable.OUTGOING_MAILS;
 import static org.assertj.core.api.Assertions.assertThat;
 import static org.mockito.ArgumentMatchers.any;
@@ -35,6 +36,8 @@ import java.util.function.Supplier;
 import org.apache.james.domainlist.api.DomainList;
 import org.apache.james.metrics.tests.RecordingMetricFactory;
 import org.apache.james.queue.api.MailQueue;
+import org.apache.mailet.Attribute;
+import org.apache.mailet.AttributeValue;
 import org.apache.mailet.Mail;
 import org.apache.mailet.base.test.FakeMail;
 import org.apache.mailet.base.test.FakeMailetConfig;
@@ -141,6 +144,7 @@ public class DeliveryRunnableTest {
         verify(mailQueue).enQueue(FakeMail.builder()
                 .name("name")
                 .attribute(DeliveryRetriesHelper.makeAttribute(1))
+                .attribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)))
                 .state(Mail.ERROR)
                 .lastUpdated(FIXED_DATE)
                 .build(),
@@ -163,6 +167,7 @@ public class DeliveryRunnableTest {
         verify(mailQueue).enQueue(FakeMail.builder()
                 .name("name")
                 .attribute(DeliveryRetriesHelper.makeAttribute(3))
+                .attribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)))
                 .state(Mail.ERROR)
                 .lastUpdated(FIXED_DATE)
                 .build(),
@@ -185,6 +190,7 @@ public class DeliveryRunnableTest {
         verify(mailQueue).enQueue(FakeMail.builder()
                 .name("name")
                 .attribute(DeliveryRetriesHelper.makeAttribute(5))
+                .attribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)))
                 .state(Mail.ERROR)
                 .lastUpdated(FIXED_DATE)
                 .build(),
@@ -238,6 +244,7 @@ public class DeliveryRunnableTest {
         verify(mailQueue).enQueue(FakeMail.builder()
                 .name("name")
                 .attribute(DeliveryRetriesHelper.makeAttribute(1))
+                .attribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)))
                 .state(Mail.ERROR)
                 .lastUpdated(FIXED_DATE)
                 .build(),


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


[james-project] 02/05: JAMES-3295 Matcher for temporary delivery failure

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

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

commit c312298fc9f21fd777780cf065f0d07aedbb11fe
Author: Rene Cordier <rc...@linagora.com>
AuthorDate: Thu Jul 9 10:31:13 2020 +0700

    JAMES-3295 Matcher for temporary delivery failure
---
 .../matchers/IsRemoteDeliveryTemporaryError.java   |  53 +++++++++++
 .../IsRemoteDeliveryTemporaryErrorTest.java        | 101 +++++++++++++++++++++
 2 files changed, 154 insertions(+)

diff --git a/server/mailet/mailets/src/main/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryError.java b/server/mailet/mailets/src/main/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryError.java
new file mode 100644
index 0000000..17ea992
--- /dev/null
+++ b/server/mailet/mailets/src/main/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryError.java
@@ -0,0 +1,53 @@
+/****************************************************************
+ * 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.transport.matchers;
+
+import static org.apache.james.transport.mailets.remote.delivery.Bouncer.IS_DELIVERY_PERMANENT_ERROR;
+
+import java.util.Collection;
+import java.util.function.Predicate;
+
+import org.apache.james.core.MailAddress;
+import org.apache.mailet.AttributeUtils;
+import org.apache.mailet.Mail;
+import org.apache.mailet.base.GenericMatcher;
+
+import com.google.common.collect.ImmutableList;
+
+/**
+ * <p>
+ * Checks if the mail has a temporary remote delivery failure attribute set to false (meaning it's a temporary error)
+ * </p>
+ *
+ * Example:
+ *
+ * <pre><code>
+ * &lt;mailet match=&quot;IsRemoteDeliveryTemporaryError&quot; class=&quot;&lt;any-class&gt;&quot;/&gt;
+ * </code></pre>
+ */
+public class IsRemoteDeliveryTemporaryError extends GenericMatcher {
+    @Override
+    public Collection<MailAddress> match(Mail mail) {
+        return AttributeUtils.getValueAndCastFromMail(mail, IS_DELIVERY_PERMANENT_ERROR, Boolean.class)
+            .filter(Predicate.isEqual(false))
+            .map(any -> mail.getRecipients())
+            .orElse(ImmutableList.of());
+    }
+}
diff --git a/server/mailet/mailets/src/test/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryErrorTest.java b/server/mailet/mailets/src/test/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryErrorTest.java
new file mode 100644
index 0000000..4a078d9
--- /dev/null
+++ b/server/mailet/mailets/src/test/java/org/apache/james/transport/matchers/IsRemoteDeliveryTemporaryErrorTest.java
@@ -0,0 +1,101 @@
+/****************************************************************
+ * 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.transport.matchers;
+
+import static org.apache.james.transport.mailets.remote.delivery.Bouncer.IS_DELIVERY_PERMANENT_ERROR;
+import static org.apache.mailet.base.MailAddressFixture.RECIPIENT1;
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.util.Collection;
+
+import javax.mail.MessagingException;
+
+import org.apache.james.core.MailAddress;
+import org.apache.mailet.Attribute;
+import org.apache.mailet.AttributeValue;
+import org.apache.mailet.Mail;
+import org.apache.mailet.base.test.FakeMail;
+import org.apache.mailet.base.test.FakeMatcherConfig;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+public class IsRemoteDeliveryTemporaryErrorTest {
+    private IsRemoteDeliveryTemporaryError testee;
+
+    private Mail createMail() throws MessagingException {
+        return FakeMail.builder()
+            .name("test-message")
+            .recipient(RECIPIENT1)
+            .build();
+    }
+
+    @BeforeEach
+    void setUp() throws Exception {
+        testee = new IsRemoteDeliveryTemporaryError();
+
+        FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
+            .matcherName("IsRemoteDeliveryTemporaryError")
+            .build();
+
+        testee.init(matcherConfig);
+    }
+
+    @Test
+    void shouldMatchWhenAttributeIsFalse() throws Exception {
+        Mail mail = createMail();
+        mail.setAttribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(false)));
+
+        Collection<MailAddress> actual = testee.match(mail);
+
+        assertThat(actual).containsOnly(RECIPIENT1);
+    }
+
+    @Test
+    void shouldNotMatchWhenAttributeIsTrue() throws Exception {
+        Mail mail = createMail();
+        mail.setAttribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(true)));
+
+        Collection<MailAddress> actual = testee.match(mail);
+
+        assertThat(actual).isEmpty();
+    }
+
+    @Test
+    void shouldNotMatchWhenAttributeIsMissing() throws Exception {
+        Mail mail = createMail();
+
+        Collection<MailAddress> actual = testee.match(mail);
+
+        assertThat(actual).isEmpty();
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"abc", "1"})
+    void shouldNotMatchWhenAttributeIsInvalid(String value) throws Exception {
+        Mail mail = createMail();
+        mail.setAttribute(new Attribute(IS_DELIVERY_PERMANENT_ERROR, AttributeValue.of(value)));
+
+        Collection<MailAddress> actual = testee.match(mail);
+
+        assertThat(actual).isEmpty();
+    }
+}


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


[james-project] 04/05: JAMES-3295 add a test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery in RemoteDeliveryErrorHandlingTest

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

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

commit 5ef6acc9764d40b009796a48a829a687931ec8cf
Author: Rene Cordier <rc...@linagora.com>
AuthorDate: Thu Jul 9 11:53:32 2020 +0700

    JAMES-3295 add a test remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery in RemoteDeliveryErrorHandlingTest
---
 .../mailets/RemoteDeliveryErrorHandlingTest.java      | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)

diff --git a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
index d7519e7..690e359 100644
--- a/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
+++ b/server/mailet/integration-testing/src/test/java/org/apache/james/transport/mailets/RemoteDeliveryErrorHandlingTest.java
@@ -57,6 +57,7 @@ import org.apache.james.utils.WebAdminGuiceProbe;
 import org.apache.james.webadmin.WebAdminUtils;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.extension.RegisterExtension;
 import org.junit.jupiter.api.io.TempDir;
@@ -69,9 +70,11 @@ public class RemoteDeliveryErrorHandlingTest {
     private static final String FROM = "from@" + DEFAULT_DOMAIN;
     private static final String RECIPIENT_DOMAIN = "test.com";
     private static final String RECIPIENT = "touser@" + RECIPIENT_DOMAIN;
+    private static final String RECIPIENT2 = "accident@" + RECIPIENT_DOMAIN;
     private static final String LOCALHOST = "localhost";
     private static final MailRepositoryUrl REMOTE_DELIVERY_TEMPORARY_ERROR_REPOSITORY = MailRepositoryUrl.from("memory://var/mail/error/remote-delivery/temporary");
     private static final MailRepositoryUrl REMOTE_DELIVERY_PERMANENT_ERROR_REPOSITORY = MailRepositoryUrl.from("memory://var/mail/error/remote-delivery/permanent");
+    private static final MailRepositoryUrl ERROR_REPOSITORY = MailRepositoryUrl.from("memory://var/mail/error/");
     private static final Integer MAX_EXECUTIONS = 2;
 
     @RegisterExtension
@@ -257,4 +260,20 @@ public class RemoteDeliveryErrorHandlingTest {
                 .getRepositoryMailCount(REMOTE_DELIVERY_PERMANENT_ERROR_REPOSITORY))
                 .isEqualTo(1));
     }
+
+    @Test
+    @Disabled("JAMES-3295 we need to add some more mailets to prove that if a mail has no temporary and permanent"
+        + "failure, it can be treated differently")
+    void remoteDeliveryErrorHandlingShouldIgnoreMailsNotTransitingByRemoteDelivery(SMTPMessageSender smtpMessageSender) throws Exception {
+        // When we relay a mail where some unexpected accident happens
+        smtpMessageSender.connect(LOCALHOST, jamesServer.getProbe(SmtpGuiceProbe.class).getSmtpPort())
+            .authenticate(FROM, PASSWORD)
+            .sendMessage(FROM, RECIPIENT2);
+
+        // Then mail should be stored in error repository
+        awaitAtMostOneMinute
+            .untilAsserted(() -> assertThat(jamesServer.getProbe(MailRepositoryProbeImpl.class)
+                .getRepositoryMailCount(ERROR_REPOSITORY))
+                .isEqualTo(1));
+    }
 }


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


[james-project] 01/05: JAMES-3295 handle configuration AtMostMatcher with differents config

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

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

commit 3c786721dff6fa9ecc61e2fb141f84369eafda27
Author: duc91 <du...@gmail.com>
AuthorDate: Wed Jul 8 15:57:50 2020 +0700

    JAMES-3295 handle configuration AtMostMatcher with differents config
---
 .../apache/james/transport/matchers/AtMost.java    |  57 +++++-
 .../james/transport/matchers/AtMostTest.java       | 221 ++++++++++++++-------
 2 files changed, 196 insertions(+), 82 deletions(-)

diff --git a/mailet/standard/src/main/java/org/apache/james/transport/matchers/AtMost.java b/mailet/standard/src/main/java/org/apache/james/transport/matchers/AtMost.java
index f3fee35..6f5b72a 100644
--- a/mailet/standard/src/main/java/org/apache/james/transport/matchers/AtMost.java
+++ b/mailet/standard/src/main/java/org/apache/james/transport/matchers/AtMost.java
@@ -20,10 +20,12 @@
 package org.apache.james.transport.matchers;
 
 import java.util.Collection;
+import java.util.List;
 import java.util.Optional;
 
 import javax.mail.MessagingException;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.james.core.MailAddress;
 import org.apache.mailet.Attribute;
 import org.apache.mailet.AttributeName;
@@ -33,39 +35,80 @@ import org.apache.mailet.Mail;
 import org.apache.mailet.base.GenericMatcher;
 import org.apache.mailet.base.MailetUtil;
 
+import com.google.common.base.Preconditions;
+import com.google.common.base.Splitter;
 import com.google.common.collect.ImmutableList;
 
 /**
  * Checks that a mail did at most X executions on a specific operation.
  *
- * If no executions have been performed previously, it sets up an attribute `AT_MOST_EXECUTIONS`
- * in the mail that will be incremented every time the check succeeds.
+ * <p> If no executions have been performed previously for Y attribute, it will be set up.</p>
+ * <p> In the mail, every time the check succeeds, its counter will be incremented by one.
+ * The check fails when the defined X limit is reached.</p>
  *
- * The check fails when the defined X limit is reached.
+ * <ul>
+ * <li>X - count of how many times a specific operation is performed</li>
+ * <li>Y - name of attribute represented for specific operation executions, default value is: <i>AT_MOST_EXECUTIONS</i></li>
+ * </ul>
  *
  * <p>The example below will match a mail with at most 3 executions on the mailet</p>
+ * with attribute name <i>AT_MOST_EXECUTIONS</i></p>
  *
  * <pre><code>
- * &lt;mailet match=&quot;AtMost=3&quot; class=&quot;&lt;any-class&gt;&quot;&gt;
+ * &lt;mailet match=&quot;AtMost=AT_MOST_EXECUTIONS:3&quot; class=&quot;&lt;any-class&gt;&quot;&gt;
  * &lt;/mailet&gt;
  * </code></pre>
  */
 public class AtMost extends GenericMatcher {
     static final AttributeName AT_MOST_EXECUTIONS = AttributeName.of("AT_MOST_EXECUTIONS");
+    private static final String CONDITION_SEPARATOR = ":";
+    private static final int ONLY_CONDITION_VALUE = 1;
+    private static final int CONDITION_NAME_AND_VALUE = 2;
+
+    private AttributeName attributeName;
     private Integer atMostExecutions;
 
     @Override
     public void init() throws MessagingException {
-        this.atMostExecutions = MailetUtil.getInitParameterAsStrictlyPositiveInteger(getCondition());
+        String conditionConfig = getMatcherConfig().getCondition();
+        Preconditions.checkArgument(StringUtils.isNotBlank(conditionConfig), "MatcherConfiguration is mandatory!");
+        Preconditions.checkArgument(!conditionConfig.startsWith(CONDITION_SEPARATOR),
+            "MatcherConfiguration can not start with '%s'", CONDITION_SEPARATOR);
+
+        List<String> conditions = Splitter.on(CONDITION_SEPARATOR).splitToList(conditionConfig);
+        attributeName = parseAttribute(conditions);
+        atMostExecutions = parseAttributeValue(conditions);
+    }
+
+    private AttributeName parseAttribute(List<String> conditions) {
+        switch (conditions.size()) {
+            case ONLY_CONDITION_VALUE:
+                return AT_MOST_EXECUTIONS;
+            case CONDITION_NAME_AND_VALUE:
+                return AttributeName.of(conditions.get(0));
+            default:
+                throw new IllegalArgumentException("MatcherConfiguration format should follow: 'name:value' or 'value'");
+        }
+    }
+
+    private Integer parseAttributeValue(List<String> conditions) throws MessagingException {
+        switch (conditions.size()) {
+            case ONLY_CONDITION_VALUE:
+                return MailetUtil.getInitParameterAsStrictlyPositiveInteger(conditions.get(0));
+            case CONDITION_NAME_AND_VALUE:
+                return MailetUtil.getInitParameterAsStrictlyPositiveInteger(conditions.get(1));
+            default:
+                throw new IllegalArgumentException("MatcherConfiguration format should follow: 'name:value' or 'value'");
+        }
     }
 
     @Override
     public Collection<MailAddress> match(Mail mail) throws MessagingException {
-        return AttributeUtils.getValueAndCastFromMail(mail, AT_MOST_EXECUTIONS, Integer.class)
+        return AttributeUtils.getValueAndCastFromMail(mail, attributeName, Integer.class)
             .or(() -> Optional.of(0))
             .filter(executions -> executions < atMostExecutions)
             .map(executions -> {
-                mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(executions + 1)));
+                mail.setAttribute(new Attribute(attributeName, AttributeValue.of(executions + 1)));
                 return mail.getRecipients();
             })
             .orElse(ImmutableList.of());
diff --git a/mailet/standard/src/test/java/org/apache/james/transport/matchers/AtMostTest.java b/mailet/standard/src/test/java/org/apache/james/transport/matchers/AtMostTest.java
index 3be5f12..355cd08 100644
--- a/mailet/standard/src/test/java/org/apache/james/transport/matchers/AtMostTest.java
+++ b/mailet/standard/src/test/java/org/apache/james/transport/matchers/AtMostTest.java
@@ -30,12 +30,14 @@ import javax.mail.MessagingException;
 
 import org.apache.james.core.MailAddress;
 import org.apache.mailet.Attribute;
+import org.apache.mailet.AttributeName;
 import org.apache.mailet.AttributeValue;
 import org.apache.mailet.Mail;
 import org.apache.mailet.base.test.FakeMail;
 import org.apache.mailet.base.test.FakeMatcherConfig;
 import org.assertj.core.api.SoftAssertions;
 import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Nested;
 import org.junit.jupiter.api.Test;
 
 import com.github.fge.lambdas.Throwing;
@@ -59,100 +61,169 @@ class AtMostTest {
             .matcherName("AtMost")
             .condition(CONDITION)
             .build();
-
         matcher.init(matcherConfig);
     }
 
-    @Test
-    void shouldMatchWhenAttributeNotSet() throws MessagingException {
-        Mail mail = createMail();
-
-        Collection<MailAddress> actual = matcher.match(mail);
-
-        assertThat(actual).containsOnly(RECIPIENT1);
+    @Nested
+    class InvalidConditionConfigurationTest {
+        @Test
+        void shouldThrowWhenMatchersConfigWithoutConditionValue() {
+            assertThatThrownBy(() -> new AtMost().init(FakeMatcherConfig.builder()
+                    .matcherName("NoValueMatcher")
+                    .condition("randomName:")
+                    .build()))
+                .isInstanceOf(MessagingException.class);
+        }
+
+        @Test
+        void shouldThrowWhenMatchersConfigWithInvalidConditionValue() {
+            assertThatThrownBy(() -> new AtMost().init(FakeMatcherConfig.builder()
+                    .matcherName("NoValueMatcher")
+                    .condition("value")
+                    .build()))
+                .isInstanceOf(MessagingException.class);
+        }
+
+        @Test
+        void shouldThrowWhenMatchersConfigWithoutConditionName() {
+            assertThatThrownBy(() -> new AtMost().init(FakeMatcherConfig.builder()
+                    .matcherName("NoValueMatcher")
+                    .condition(":3")
+                    .build()))
+                .isInstanceOf(IllegalArgumentException.class);
+        }
+
+        @Test
+        void shouldThrowWhenMatchersConfigNameAsSpace() {
+            assertThatThrownBy(() -> new AtMost().init(FakeMatcherConfig.builder()
+                    .matcherName("NoValueMatcher")
+                    .condition("  :  ")
+                    .build()))
+                .isInstanceOf(MessagingException.class);
+        }
+
+
+        @Test
+        void shouldThrowWithEmptyCondition() {
+            FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
+                .matcherName("AtMost")
+                .build();
+
+            assertThatThrownBy(() -> matcher.init(matcherConfig))
+                .isInstanceOf(IllegalArgumentException.class);
+        }
+
+        @Test
+        void shouldThrowWithNegativeCondition() {
+            FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
+                .matcherName("AtMost")
+                .condition("-1")
+                .build();
+
+            assertThatThrownBy(() -> matcher.init(matcherConfig))
+                .isInstanceOf(MessagingException.class);
+        }
+
+        @Test
+        void shouldThrowWithConditionToZero() {
+            FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
+                .matcherName("AtMost")
+                .condition("0")
+                .build();
+
+            assertThatThrownBy(() -> matcher.init(matcherConfig))
+                .isInstanceOf(MessagingException.class);
+        }
     }
 
-    @Test
-    void shouldMatchWhenNoRetries() throws MessagingException {
-        Mail mail = createMail();
-        mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(0)));
-
-        Collection<MailAddress> actual = matcher.match(mail);
-
-        assertThat(actual).containsOnly(RECIPIENT1);
+    @Nested
+    class MultipleMatchersConfigurationTest {
+        private AtMost atMost2;
+        private AtMost atMost3;
+
+        @BeforeEach
+        void setup() throws MessagingException {
+            this.atMost2 = new AtMost();
+            atMost2.init(
+                FakeMatcherConfig.builder()
+                    .matcherName("AtMost")
+                    .condition("AtMost2:2")
+                    .build());
+
+            this.atMost3 = new AtMost();
+            atMost3.init(
+                FakeMatcherConfig.builder()
+                    .matcherName("AtMost")
+                    .condition("AtMost3:2")
+                    .build());
+        }
+
+        @Test
+        void matchersShouldStopWhenAMatcherReachedLimit() throws MessagingException {
+            Mail mail1 = createMail();
+
+            SoftAssertions.assertSoftly(Throwing.consumer(
+                softly -> {
+                    softly.assertThat(atMost2.match(mail1)).containsOnly(RECIPIENT1);
+                    softly.assertThat(atMost2.match(mail1)).containsOnly(RECIPIENT1);
+                    softly.assertThat(atMost2.match(mail1)).isEmpty();
+                    softly.assertThat(atMost3.match(mail1)).containsOnly(RECIPIENT1);
+                    softly.assertThat(atMost3.match(mail1)).containsOnly(RECIPIENT1);
+                    softly.assertThat(atMost3.match(mail1)).isEmpty();
+                }));
+        }
     }
 
-    @Test
-    void shouldNotMatchWhenOverAtMost() throws MessagingException {
-        Mail mail = createMail();
-        mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(3)));
+    @Nested
+    class SingleMatcherConfigurationTest {
+        @Test
+        void shouldMatchWhenAttributeNotSet() throws MessagingException {
+            Mail mail = createMail();
 
-        Collection<MailAddress> actual = matcher.match(mail);
+            Collection<MailAddress> actual = matcher.match(mail);
 
-        assertThat(actual).isEmpty();
-    }
+            assertThat(actual).containsOnly(RECIPIENT1);
+        }
 
-    @Test
-    void shouldNotMatchWhenEqualToAtMost() throws MessagingException {
-        Mail mail = createMail();
-        mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(2)));
+        @Test
+        void shouldMatchWhenNoRetries() throws MessagingException {
+            Mail mail = createMail();
+            mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(0)));
 
-        Collection<MailAddress> actual = matcher.match(mail);
+            Collection<MailAddress> actual = matcher.match(mail);
 
-        assertThat(actual).isEmpty();
-    }
+            assertThat(actual).containsOnly(RECIPIENT1);
+        }
 
-    @Test
-    void shouldThrowWithEmptyCondition() {
-        FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
-            .matcherName("AtMost")
-            .build();
+        @Test
+        void shouldNotMatchWhenOverAtMost() throws MessagingException {
+            Mail mail = createMail();
+            mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(3)));
 
-        assertThatThrownBy(() -> matcher.init(matcherConfig))
-            .isInstanceOf(MessagingException.class);
-    }
+            Collection<MailAddress> actual = matcher.match(mail);
 
-    @Test
-    void shouldThrowWithInvalidCondition() {
-        FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
-            .matcherName("AtMost")
-            .condition("invalid")
-            .build();
-
-        assertThatThrownBy(() -> matcher.init(matcherConfig))
-            .isInstanceOf(MessagingException.class);
-    }
-
-    @Test
-    void shouldThrowWithNegativeCondition() {
-        FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
-            .matcherName("AtMost")
-            .condition("-1")
-            .build();
+            assertThat(actual).isEmpty();
+        }
 
-        assertThatThrownBy(() -> matcher.init(matcherConfig))
-            .isInstanceOf(MessagingException.class);
-    }
+        @Test
+        void shouldNotMatchWhenEqualToAtMost() throws MessagingException {
+            Mail mail = createMail();
+            mail.setAttribute(new Attribute(AT_MOST_EXECUTIONS, AttributeValue.of(2)));
 
-    @Test
-    void shouldThrowWithConditionToZero() {
-        FakeMatcherConfig matcherConfig = FakeMatcherConfig.builder()
-            .matcherName("AtMost")
-            .condition("0")
-            .build();
+            Collection<MailAddress> actual = matcher.match(mail);
 
-        assertThatThrownBy(() -> matcher.init(matcherConfig))
-            .isInstanceOf(MessagingException.class);
-    }
+            assertThat(actual).isEmpty();
+        }
 
-    @Test
-    void shouldMatchUntilOverAtMost() throws MessagingException {
-        Mail mail = createMail();
+        @Test
+        void shouldMatchUntilOverAtMost() throws MessagingException {
+            Mail mail = createMail();
 
-        SoftAssertions.assertSoftly(Throwing.consumer(softly -> {
-            softly.assertThat(matcher.match(mail)).describedAs("First execution").contains(RECIPIENT1);
-            softly.assertThat(matcher.match(mail)).describedAs("Second execution").contains(RECIPIENT1);
-            softly.assertThat(matcher.match(mail)).describedAs("Third execution").isEmpty();
-        }));
+            SoftAssertions.assertSoftly(Throwing.consumer(softly -> {
+                softly.assertThat(matcher.match(mail)).describedAs("First execution").contains(RECIPIENT1);
+                softly.assertThat(matcher.match(mail)).describedAs("Second execution").contains(RECIPIENT1);
+                softly.assertThat(matcher.match(mail)).describedAs("Third execution").isEmpty();
+            }));
+        }
     }
 }


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