You are viewing a plain text version of this content. The canonical link for it is here.
Posted to server-dev@james.apache.org by bt...@apache.org on 2020/04/22 02:31:50 UTC

[james-project] 10/37: JAMES-2997 step #8 Stop relying on Attachment byte array in AttachmentMapper::storeAttachmentsForMessage

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 431a8e128bf972f4f78792ae93f59f5c1ac30b86
Author: Benoit Tellier <bt...@linagora.com>
AuthorDate: Fri Jan 17 13:41:34 2020 +0700

    JAMES-2997 step #8 Stop relying on Attachment byte array in AttachmentMapper::storeAttachmentsForMessage
    
    This forced me to review attachment write path:
     - AttachmentId needs to be allocated by the attachment mapper
     - Thus we can not return MessageAttachment (requiring an attachmentId) within
     Message parser
     - Attachments needs to be persisted first, then the message is persisted.
    
    Along the way improvements:
     - Rely on casting to avoid byte reallocation upon reading content in MessageParser
     - Avoid parsing attachment for backends not storing them
---
 mailbox/api/pom.xml                                |   1 -
 .../james/mailbox/model/MessageAttachment.java     |   5 +
 .../james/mailbox/model/ParsedAttachment.java      | 130 +++++++++++++++++++++
 .../mailbox/cassandra/CassandraMessageManager.java |  16 +--
 .../cassandra/mail/CassandraAttachmentMapper.java  |  19 +--
 .../model/openjpa/AbstractJPAMailboxMessage.java   |   9 +-
 .../mailbox/maildir/mail/model/MaildirMessage.java |   9 +-
 .../mailbox/inmemory/InMemoryMessageManager.java   |  16 +--
 .../inmemory/mail/InMemoryAttachmentMapper.java    |  32 ++++-
 .../james/mailbox/store/StoreMessageManager.java   |  26 ++---
 .../james/mailbox/store/mail/AttachmentMapper.java |   4 +-
 .../store/mail/model/impl/MessageParser.java       |  37 +++---
 .../model/MessageWithAttachmentMapperTest.java     |  66 +++++------
 13 files changed, 264 insertions(+), 106 deletions(-)

diff --git a/mailbox/api/pom.xml b/mailbox/api/pom.xml
index 4706a0e..7ce79e2 100644
--- a/mailbox/api/pom.xml
+++ b/mailbox/api/pom.xml
@@ -82,7 +82,6 @@
         <dependency>
             <groupId>commons-io</groupId>
             <artifactId>commons-io</artifactId>
-            <scope>test</scope>
         </dependency>
         <dependency>
             <groupId>javax.inject</groupId>
diff --git a/mailbox/api/src/main/java/org/apache/james/mailbox/model/MessageAttachment.java b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MessageAttachment.java
index fc37b03..8e6b407 100644
--- a/mailbox/api/src/main/java/org/apache/james/mailbox/model/MessageAttachment.java
+++ b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MessageAttachment.java
@@ -55,6 +55,11 @@ public class MessageAttachment {
             return this;
         }
 
+        public Builder name(Optional<String> name) {
+            this.name = name;
+            return this;
+        }
+
         public Builder cid(Optional<Cid> cid) {
             Preconditions.checkNotNull(cid);
             this.cid = cid;
diff --git a/mailbox/api/src/main/java/org/apache/james/mailbox/model/ParsedAttachment.java b/mailbox/api/src/main/java/org/apache/james/mailbox/model/ParsedAttachment.java
new file mode 100644
index 0000000..649b034
--- /dev/null
+++ b/mailbox/api/src/main/java/org/apache/james/mailbox/model/ParsedAttachment.java
@@ -0,0 +1,130 @@
+/****************************************************************
+ * 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.mailbox.model;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Optional;
+
+import org.apache.commons.io.IOUtils;
+
+public class ParsedAttachment {
+    interface Builder {
+        @FunctionalInterface
+        interface RequireContentType {
+            RequireContent contentType(String contentType);
+        }
+
+        @FunctionalInterface
+        interface RequireContent {
+            RequireName content(InputStream stream);
+        }
+
+        @FunctionalInterface
+        interface RequireName {
+            RequireCid name(Optional<String> name);
+
+            default RequireCid name(String name) {
+                return name(Optional.of(name));
+            }
+
+            default RequireCid noName() {
+                return name(Optional.empty());
+            }
+        }
+
+        @FunctionalInterface
+        interface RequireCid {
+            RequireIsInline cid(Optional<Cid> cid);
+
+            default RequireIsInline cid(Cid cid) {
+                return cid(Optional.of(cid));
+            }
+
+            default RequireIsInline noCid() {
+                return cid(Optional.empty());
+            }
+        }
+
+        @FunctionalInterface
+        interface RequireIsInline {
+            ParsedAttachment inline(boolean isInline);
+
+            default ParsedAttachment inline() {
+                return inline(true);
+            }
+        }
+    }
+
+    public static Builder.RequireContentType builder() {
+        return contentType -> content -> name -> cid -> isInline -> new ParsedAttachment(contentType, content, name, cid, isInline);
+    }
+
+    private final String contentType;
+    private final InputStream content;
+    private final Optional<String> name;
+    private final Optional<Cid> cid;
+    private final boolean isInline;
+
+    private ParsedAttachment(String contentType, InputStream content, Optional<String> name, Optional<Cid> cid, boolean isInline) {
+        this.contentType = contentType;
+        this.content = content;
+        this.name = name;
+        this.cid = cid;
+        this.isInline = isInline;
+    }
+
+    public String getContentType() {
+        return contentType;
+    }
+
+    public InputStream getContent() {
+        return content;
+    }
+
+    public Optional<String> getName() {
+        return name;
+    }
+
+    public Optional<Cid> getCid() {
+        return cid;
+    }
+
+    public boolean isInline() {
+        return isInline;
+    }
+
+    public MessageAttachment asMessageAttachment(AttachmentId attachmentId) {
+        try {
+            return MessageAttachment.builder()
+                .attachment(Attachment.builder()
+                        .attachmentId(attachmentId)
+                        .type(contentType)
+                        .bytes(IOUtils.toByteArray(content))
+                        .build())
+                    .name(name)
+                    .cid(cid)
+                    .isInline(isInline)
+                    .build();
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+}
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMessageManager.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMessageManager.java
index d699e7d..d3cbdde 100644
--- a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMessageManager.java
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMessageManager.java
@@ -22,6 +22,7 @@ package org.apache.james.mailbox.cassandra;
 import java.util.List;
 
 import javax.mail.Flags;
+import javax.mail.internet.SharedInputStream;
 
 import org.apache.james.mailbox.MailboxPathLocker;
 import org.apache.james.mailbox.MailboxSession;
@@ -30,18 +31,16 @@ import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.model.Mailbox;
 import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.quota.QuotaManager;
 import org.apache.james.mailbox.quota.QuotaRootResolver;
 import org.apache.james.mailbox.store.BatchSizes;
 import org.apache.james.mailbox.store.PreDeletionHooks;
 import org.apache.james.mailbox.store.StoreMessageManager;
 import org.apache.james.mailbox.store.StoreRightManager;
-import org.apache.james.mailbox.store.mail.model.MailboxMessage;
 import org.apache.james.mailbox.store.mail.model.impl.MessageParser;
 import org.apache.james.mailbox.store.search.MessageSearchIndex;
 
-import com.github.steveash.guavate.Guavate;
-
 /**
  * Cassandra implementation of {@link StoreMessageManager}
  * 
@@ -74,13 +73,10 @@ public class CassandraMessageManager extends StoreMessageManager {
     }
 
     @Override
-    protected void storeAttachment(MailboxMessage message, List<MessageAttachment> messageAttachments, MailboxSession session) throws MailboxException {
-        mapperFactory.getAttachmentMapper(session)
-            .storeAttachmentsForMessage(
-                messageAttachments.stream()
-                    .map(MessageAttachment::getAttachment)
-                    .collect(Guavate.toImmutableList()),
-                message.getMessageId());
+    protected List<MessageAttachment> storeAttachments(MessageId messageId, SharedInputStream content, MailboxSession session) throws MailboxException {
+        List<ParsedAttachment> attachments = extractAttachments(content);
+        return mapperFactory.getAttachmentMapper(session)
+            .storeAttachmentsForMessage(attachments, messageId);
     }
 
 }
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentMapper.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentMapper.java
index 534e3f1..74fe75c 100644
--- a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentMapper.java
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentMapper.java
@@ -36,7 +36,9 @@ import org.apache.james.mailbox.exception.AttachmentNotFoundException;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.model.Attachment;
 import org.apache.james.mailbox.model.AttachmentId;
+import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.store.mail.AttachmentMapper;
 import org.apache.james.util.ReactorUtils;
 import org.apache.james.util.io.SizeInputStream;
@@ -130,10 +132,10 @@ public class CassandraAttachmentMapper implements AttachmentMapper {
     }
 
     @Override
-    public void storeAttachmentsForMessage(Collection<Attachment> attachments, MessageId ownerMessageId) throws MailboxException {
-        Flux.fromIterable(attachments)
+    public List<MessageAttachment> storeAttachmentsForMessage(Collection<ParsedAttachment> parsedAttachments, MessageId ownerMessageId) throws MailboxException {
+        return Flux.fromIterable(parsedAttachments)
             .flatMap(attachment -> storeAttachmentAsync(attachment, ownerMessageId))
-            .then()
+            .collectList()
             .block();
     }
 
@@ -149,10 +151,13 @@ public class CassandraAttachmentMapper implements AttachmentMapper {
         return ownerDAO.retrieveOwners(attachmentId).collect(Guavate.toImmutableList()).block();
     }
 
-    public Mono<Void> storeAttachmentAsync(Attachment attachment, MessageId ownerMessageId) {
-        return Mono.from(blobStore.save(blobStore.getDefaultBucketName(), attachment.getBytes(), LOW_COST))
-            .map(blobId -> CassandraAttachmentDAOV2.from(attachment, blobId))
-            .flatMap(daoAttachment -> storeAttachmentWithIndex(daoAttachment, ownerMessageId));
+    private Mono<MessageAttachment> storeAttachmentAsync(ParsedAttachment parsedAttachment, MessageId ownerMessageId) {
+        AttachmentId attachmentId = AttachmentId.random();
+        SizeInputStream content = new SizeInputStream(parsedAttachment.getContent());
+        return Mono.from(blobStore.save(blobStore.getDefaultBucketName(), content, LOW_COST))
+            .map(blobId -> new DAOAttachment(attachmentId, blobId, parsedAttachment.getContentType(), content.getSize()))
+            .flatMap(daoAttachment -> storeAttachmentWithIndex(daoAttachment, ownerMessageId))
+            .then(Mono.just(parsedAttachment.asMessageAttachment(attachmentId)));
     }
 
     private Mono<Void> storeAttachmentWithIndex(DAOAttachment daoAttachment, MessageId ownerMessageId) {
diff --git a/mailbox/jpa/src/main/java/org/apache/james/mailbox/jpa/mail/model/openjpa/AbstractJPAMailboxMessage.java b/mailbox/jpa/src/main/java/org/apache/james/mailbox/jpa/mail/model/openjpa/AbstractJPAMailboxMessage.java
index cfebd40..f174fe6 100644
--- a/mailbox/jpa/src/main/java/org/apache/james/mailbox/jpa/mail/model/openjpa/AbstractJPAMailboxMessage.java
+++ b/mailbox/jpa/src/main/java/org/apache/james/mailbox/jpa/mail/model/openjpa/AbstractJPAMailboxMessage.java
@@ -48,6 +48,7 @@ import org.apache.james.mailbox.jpa.JPAId;
 import org.apache.james.mailbox.jpa.mail.model.JPAMailbox;
 import org.apache.james.mailbox.jpa.mail.model.JPAProperty;
 import org.apache.james.mailbox.jpa.mail.model.JPAUserFlag;
+import org.apache.james.mailbox.model.AttachmentId;
 import org.apache.james.mailbox.model.ComposedMessageId;
 import org.apache.james.mailbox.model.ComposedMessageIdWithMetaData;
 import org.apache.james.mailbox.model.MessageAttachment;
@@ -59,7 +60,6 @@ import org.apache.james.mailbox.store.mail.model.MailboxMessage;
 import org.apache.james.mailbox.store.mail.model.Property;
 import org.apache.james.mailbox.store.mail.model.impl.MessageParser;
 import org.apache.james.mailbox.store.mail.model.impl.PropertyBuilder;
-import org.apache.james.mime4j.MimeException;
 import org.apache.openjpa.persistence.jdbc.ElementJoinColumn;
 import org.apache.openjpa.persistence.jdbc.ElementJoinColumns;
 import org.apache.openjpa.persistence.jdbc.Index;
@@ -505,8 +505,11 @@ public abstract class AbstractJPAMailboxMessage implements MailboxMessage {
     @Override
     public List<MessageAttachment> getAttachments() {
         try {
-            return new MessageParser().retrieveAttachments(getFullContent());
-        } catch (MimeException | IOException e) {
+            return new MessageParser().retrieveAttachments(getFullContent())
+                .stream()
+                .map(attachmentMetadata -> attachmentMetadata.asMessageAttachment(AttachmentId.random()))
+                .collect(Guavate.toImmutableList());
+        } catch (IOException e) {
             throw new RuntimeException(e);
         }
     }
diff --git a/mailbox/maildir/src/main/java/org/apache/james/mailbox/maildir/mail/model/MaildirMessage.java b/mailbox/maildir/src/main/java/org/apache/james/mailbox/maildir/mail/model/MaildirMessage.java
index c92a2d9..c2a0b01 100644
--- a/mailbox/maildir/src/main/java/org/apache/james/mailbox/maildir/mail/model/MaildirMessage.java
+++ b/mailbox/maildir/src/main/java/org/apache/james/mailbox/maildir/mail/model/MaildirMessage.java
@@ -30,6 +30,7 @@ import javax.mail.util.SharedFileInputStream;
 
 import org.apache.commons.io.IOUtils;
 import org.apache.james.mailbox.maildir.MaildirMessageName;
+import org.apache.james.mailbox.model.AttachmentId;
 import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
 import org.apache.james.mailbox.store.mail.model.DefaultMessageId;
@@ -46,6 +47,7 @@ import org.apache.james.mime4j.stream.MimeConfig;
 import org.apache.james.mime4j.stream.MimeTokenStream;
 import org.apache.james.mime4j.stream.RecursionMode;
 
+import com.github.steveash.guavate.Guavate;
 import com.google.common.io.ByteStreams;
 
 public class MaildirMessage implements Message {
@@ -271,8 +273,11 @@ public class MaildirMessage implements Message {
     @Override
     public List<MessageAttachment> getAttachments() {
         try {
-            return new MessageParser().retrieveAttachments(getFullContent());
-        } catch (MimeException | IOException e) {
+            return new MessageParser().retrieveAttachments(getFullContent())
+                .stream()
+                .map(attachmentMetadata -> attachmentMetadata.asMessageAttachment(AttachmentId.random()))
+                .collect(Guavate.toImmutableList());
+        } catch (IOException e) {
             throw new RuntimeException(e);
         }
     }
diff --git a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryMessageManager.java b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryMessageManager.java
index fed0e7b..ef84c37 100644
--- a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryMessageManager.java
+++ b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryMessageManager.java
@@ -3,6 +3,7 @@ package org.apache.james.mailbox.inmemory;
 import java.util.List;
 
 import javax.mail.Flags;
+import javax.mail.internet.SharedInputStream;
 
 import org.apache.james.mailbox.MailboxPathLocker;
 import org.apache.james.mailbox.MailboxSession;
@@ -11,6 +12,7 @@ import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.model.Mailbox;
 import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.quota.QuotaManager;
 import org.apache.james.mailbox.quota.QuotaRootResolver;
 import org.apache.james.mailbox.store.BatchSizes;
@@ -18,12 +20,9 @@ import org.apache.james.mailbox.store.MailboxSessionMapperFactory;
 import org.apache.james.mailbox.store.PreDeletionHooks;
 import org.apache.james.mailbox.store.StoreMessageManager;
 import org.apache.james.mailbox.store.StoreRightManager;
-import org.apache.james.mailbox.store.mail.model.MailboxMessage;
 import org.apache.james.mailbox.store.mail.model.impl.MessageParser;
 import org.apache.james.mailbox.store.search.MessageSearchIndex;
 
-import com.github.steveash.guavate.Guavate;
-
 public class InMemoryMessageManager extends StoreMessageManager {
 
     private InMemoryMailboxSessionMapperFactory mapperFactory;
@@ -54,12 +53,9 @@ public class InMemoryMessageManager extends StoreMessageManager {
     }
 
     @Override
-    protected void storeAttachment(final MailboxMessage message, final List<MessageAttachment> messageAttachments, final MailboxSession session) throws MailboxException {
-        mapperFactory.getAttachmentMapper(session)
-            .storeAttachmentsForMessage(
-                messageAttachments.stream()
-                    .map(MessageAttachment::getAttachment)
-                    .collect(Guavate.toImmutableList()),
-                message.getMessageId());
+    protected List<MessageAttachment> storeAttachments(MessageId messageId, SharedInputStream content, MailboxSession session) throws MailboxException {
+        List<ParsedAttachment> attachments = extractAttachments(content);
+        return mapperFactory.getAttachmentMapper(session)
+            .storeAttachmentsForMessage(attachments, messageId);
     }
 }
diff --git a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/mail/InMemoryAttachmentMapper.java b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/mail/InMemoryAttachmentMapper.java
index 9112ece..331c60f 100644
--- a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/mail/InMemoryAttachmentMapper.java
+++ b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/mail/InMemoryAttachmentMapper.java
@@ -33,9 +33,13 @@ import org.apache.james.mailbox.exception.AttachmentNotFoundException;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.model.Attachment;
 import org.apache.james.mailbox.model.AttachmentId;
+import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.store.mail.AttachmentMapper;
 
+import com.github.fge.lambdas.Throwing;
+import com.github.steveash.guavate.Guavate;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
@@ -116,11 +120,29 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     }
 
     @Override
-    public void storeAttachmentsForMessage(Collection<Attachment> attachments, MessageId ownerMessageId) throws MailboxException {
-        for (Attachment attachment: attachments) {
-            attachmentsById.put(attachment.getAttachmentId(), attachment);
-            attachmentsRawContentById.put(attachment.getAttachmentId(), attachment.getBytes());
-            messageIdsByAttachmentId.put(attachment.getAttachmentId(), ownerMessageId);
+    public List<MessageAttachment> storeAttachmentsForMessage(Collection<ParsedAttachment> parsedAttachments, MessageId ownerMessageId) throws MailboxException {
+        return parsedAttachments.stream()
+            .map(Throwing.<ParsedAttachment, MessageAttachment>function(
+                typedContent -> storeAttachmentForMessage(ownerMessageId, typedContent))
+                .sneakyThrow())
+            .collect(Guavate.toImmutableList());
+    }
+
+    private MessageAttachment storeAttachmentForMessage(MessageId ownerMessageId, ParsedAttachment parsedAttachment) throws MailboxException {
+        AttachmentId attachmentId = AttachmentId.random();
+        try {
+            byte[] bytes = IOUtils.toByteArray(parsedAttachment.getContent());
+
+            attachmentsById.put(attachmentId, Attachment.builder()
+                .attachmentId(attachmentId)
+                .type(parsedAttachment.getContentType())
+                .bytes(bytes)
+                .build());
+            attachmentsRawContentById.put(attachmentId, bytes);
+            messageIdsByAttachmentId.put(attachmentId, ownerMessageId);
+            return parsedAttachment.asMessageAttachment(attachmentId);
+        } catch (IOException e) {
+            throw new MailboxException(String.format("Failed to persist attachment %s of message %s", attachmentId, ownerMessageId.serialize()), e);
         }
     }
 
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreMessageManager.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreMessageManager.java
index 1eb89a7..9f2557b 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreMessageManager.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreMessageManager.java
@@ -72,6 +72,7 @@ import org.apache.james.mailbox.model.MessageMetaData;
 import org.apache.james.mailbox.model.MessageMoves;
 import org.apache.james.mailbox.model.MessageRange;
 import org.apache.james.mailbox.model.MessageResultIterator;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.UidValidity;
 import org.apache.james.mailbox.model.UpdatedFlags;
@@ -445,15 +446,10 @@ public class StoreMessageManager implements MessageManager {
     private ComposedMessageId createAndDispatchMessage(Date internalDate, MailboxSession mailboxSession, File file, PropertyBuilder propertyBuilder, Flags flags, int bodyStartOctet) throws IOException, MailboxException {
         try (SharedFileInputStream contentIn = new SharedFileInputStream(file)) {
             final int size = (int) file.length();
-
-            final List<MessageAttachment> attachments = extractAttachments(contentIn);
-
-            final MailboxMessage message = createMessage(internalDate, size, bodyStartOctet, contentIn, flags, propertyBuilder, attachments);
-
             new QuotaChecker(quotaManager, quotaRootResolver, mailbox).tryAddition(1, size);
 
             return locker.executeWithLock(getMailboxPath(), () -> {
-                MessageMetaData data = appendMessageToStore(message, attachments, mailboxSession);
+                MessageMetaData data = appendMessageToStore(internalDate, size, bodyStartOctet, contentIn, flags, propertyBuilder, mailboxSession);
 
                 Mailbox mailbox = getMailboxEntity();
 
@@ -461,7 +457,7 @@ public class StoreMessageManager implements MessageManager {
                     .randomEventId()
                     .mailboxSession(mailboxSession)
                     .mailbox(mailbox)
-                    .addMetaData(message.metaData())
+                    .addMetaData(data)
                     .build(),
                     new MailboxIdRegistrationKey(mailbox.getMailboxId()))
                     .subscribeOn(Schedulers.elastic())
@@ -500,9 +496,9 @@ public class StoreMessageManager implements MessageManager {
         return propertyBuilder;
     }
 
-    private List<MessageAttachment> extractAttachments(SharedFileInputStream contentIn) {
+    protected List<ParsedAttachment> extractAttachments(SharedInputStream contentIn) {
         try {
-            return messageParser.retrieveAttachments(contentIn);
+            return messageParser.retrieveAttachments(contentIn.newStream(0, -1));
         } catch (Exception e) {
             LOG.warn("Error while parsing mail's attachments: {}", e.getMessage(), e);
             return ImmutableList.of();
@@ -672,17 +668,19 @@ public class StoreMessageManager implements MessageManager {
         }, MailboxPathLocker.LockType.Write);
     }
 
-    protected MessageMetaData appendMessageToStore(final MailboxMessage message, final List<MessageAttachment> messageAttachments, MailboxSession session) throws MailboxException {
-        final MessageMapper messageMapper = mapperFactory.getMessageMapper(session);
+    private MessageMetaData appendMessageToStore(Date internalDate, int size, int bodyStartOctet, SharedInputStream content, Flags flags, PropertyBuilder propertyBuilder, MailboxSession session) throws MailboxException {
+        MessageMapper messageMapper = mapperFactory.getMessageMapper(session);
+        MessageId messageId = messageIdFactory.generate();
 
         return mapperFactory.getMessageMapper(session).execute(() -> {
-            storeAttachment(message, messageAttachments, session);
+            List<MessageAttachment> attachments = storeAttachments(messageId, content, session);
+            MailboxMessage message = createMessage(internalDate, size, bodyStartOctet, content, flags, propertyBuilder, attachments);
             return messageMapper.add(getMailboxEntity(), message);
         });
     }
 
-    protected void storeAttachment(final MailboxMessage message, final List<MessageAttachment> messageAttachments, final MailboxSession session) throws MailboxException {
-
+    protected List<MessageAttachment> storeAttachments(MessageId messageId, SharedInputStream content, MailboxSession session) throws MailboxException {
+        return ImmutableList.of();
     }
 
     @Override
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/AttachmentMapper.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/AttachmentMapper.java
index 6b5493b..196eb91 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/AttachmentMapper.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/AttachmentMapper.java
@@ -29,7 +29,9 @@ import org.apache.james.mailbox.exception.AttachmentNotFoundException;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.model.Attachment;
 import org.apache.james.mailbox.model.AttachmentId;
+import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.store.transaction.Mapper;
 import org.reactivestreams.Publisher;
 
@@ -43,7 +45,7 @@ public interface AttachmentMapper extends Mapper {
 
     Publisher<Attachment> storeAttachmentForOwner(String contentType, InputStream attachmentContent, Username owner);
 
-    void storeAttachmentsForMessage(Collection<Attachment> attachments, MessageId ownerMessageId) throws MailboxException;
+    List<MessageAttachment> storeAttachmentsForMessage(Collection<ParsedAttachment> attachments, MessageId ownerMessageId) throws MailboxException;
 
     Collection<MessageId> getRelatedMessageIds(AttachmentId attachmentId) throws MailboxException;
 
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/model/impl/MessageParser.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/model/impl/MessageParser.java
index 50b849d..39a3c75 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/model/impl/MessageParser.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/model/impl/MessageParser.java
@@ -19,6 +19,7 @@
 
 package org.apache.james.mailbox.store.mail.model.impl;
 
+import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
@@ -27,15 +28,14 @@ import java.util.Locale;
 import java.util.Optional;
 import java.util.stream.Stream;
 
-import org.apache.james.mailbox.model.Attachment;
 import org.apache.james.mailbox.model.Cid;
-import org.apache.james.mailbox.model.MessageAttachment;
-import org.apache.james.mime4j.MimeException;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mime4j.codec.DecodeMonitor;
 import org.apache.james.mime4j.dom.Body;
 import org.apache.james.mime4j.dom.Entity;
 import org.apache.james.mime4j.dom.Message;
 import org.apache.james.mime4j.dom.Multipart;
+import org.apache.james.mime4j.dom.SingleBody;
 import org.apache.james.mime4j.dom.field.ContentDispositionField;
 import org.apache.james.mime4j.dom.field.ContentIdField;
 import org.apache.james.mime4j.dom.field.ContentTypeField;
@@ -76,7 +76,7 @@ public class MessageParser {
             .unwrap();
     }
 
-    public List<MessageAttachment> retrieveAttachments(InputStream fullContent) throws MimeException, IOException {
+    public List<ParsedAttachment> retrieveAttachments(InputStream fullContent) throws IOException {
         DefaultMessageBuilder defaultMessageBuilder = new DefaultMessageBuilder();
         defaultMessageBuilder.setMimeEntityConfig(MimeConfig.PERMISSIVE);
         defaultMessageBuilder.setDecodeMonitor(DecodeMonitor.SILENT);
@@ -99,13 +99,13 @@ public class MessageParser {
         }
     }
 
-    private Stream<MessageAttachment> listAttachments(Multipart multipart, Context context) {
+    private Stream<ParsedAttachment> listAttachments(Multipart multipart, Context context) {
         return multipart.getBodyParts()
             .stream()
             .flatMap(entity -> listAttachments(entity, context));
     }
 
-    private Stream<MessageAttachment> listAttachments(Entity entity, Context context) {
+    private Stream<ParsedAttachment> listAttachments(Entity entity, Context context) {
         if (isMultipart(entity)) {
             return listAttachments((Multipart) entity.getBody(), Context.fromEntity(entity));
         }
@@ -121,7 +121,7 @@ public class MessageParser {
         return Stream.empty();
     }
 
-    private MessageAttachment retrieveAttachment(Entity entity) throws IOException {
+    private ParsedAttachment retrieveAttachment(Entity entity) throws IOException {
         Optional<ContentTypeField> contentTypeField = getContentTypeField(entity);
         Optional<ContentDispositionField> contentDispositionField = getContentDispositionField(entity);
         Optional<String> contentType = contentType(contentTypeField);
@@ -129,15 +129,12 @@ public class MessageParser {
         Optional<Cid> cid = cid(readHeader(entity, CONTENT_ID, ContentIdField.class));
         boolean isInline = isInline(readHeader(entity, CONTENT_DISPOSITION, ContentDispositionField.class)) && cid.isPresent();
 
-        return MessageAttachment.builder()
-                .attachment(Attachment.builder()
-                    .bytes(getBytes(entity.getBody()))
-                    .type(contentType.orElse(DEFAULT_CONTENT_TYPE))
-                    .build())
-                .name(name.orElse(null))
-                .cid(cid.orElse(null))
-                .isInline(isInline)
-                .build();
+        return ParsedAttachment.builder()
+                .contentType(contentType.orElse(DEFAULT_CONTENT_TYPE))
+                .content(getContent(entity.getBody()))
+                .name(name)
+                .cid(cid)
+                .inline(isInline);
     }
 
     private <T extends ParsedField> Optional<T> readHeader(Entity entity, String headerName, Class<T> clazz) {
@@ -221,11 +218,15 @@ public class MessageParser {
         return readHeader(part, CONTENT_ID, ContentIdField.class).isPresent();
     }
 
-    private byte[] getBytes(Body body) throws IOException {
+    private InputStream getContent(Body body) throws IOException {
+        if (body instanceof SingleBody) {
+            SingleBody singleBody = (SingleBody) body;
+            return singleBody.getInputStream();
+        }
         DefaultMessageWriter messageWriter = new DefaultMessageWriter();
         ByteArrayOutputStream out = new ByteArrayOutputStream();
         messageWriter.writeBody(body, out);
-        return out.toByteArray();
+        return new ByteArrayInputStream(out.toByteArray());
     }
 
     private enum Context {
diff --git a/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/MessageWithAttachmentMapperTest.java b/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/MessageWithAttachmentMapperTest.java
index f10a0b6..f8d24ba 100644
--- a/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/MessageWithAttachmentMapperTest.java
+++ b/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/MessageWithAttachmentMapperTest.java
@@ -22,7 +22,9 @@ package org.apache.james.mailbox.store.mail.model;
 import static org.apache.james.mailbox.store.mail.model.MessageAssert.assertThat;
 import static org.assertj.core.api.Assertions.assertThat;
 
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
+import java.nio.charset.StandardCharsets;
 import java.util.Date;
 import java.util.Iterator;
 import java.util.List;
@@ -32,8 +34,6 @@ import javax.mail.util.SharedByteArrayInputStream;
 
 import org.apache.james.core.Username;
 import org.apache.james.mailbox.exception.MailboxException;
-import org.apache.james.mailbox.model.Attachment;
-import org.apache.james.mailbox.model.AttachmentId;
 import org.apache.james.mailbox.model.Cid;
 import org.apache.james.mailbox.model.Mailbox;
 import org.apache.james.mailbox.model.MailboxPath;
@@ -41,6 +41,7 @@ import org.apache.james.mailbox.model.MessageAttachment;
 import org.apache.james.mailbox.model.MessageId;
 import org.apache.james.mailbox.model.MessageRange;
 import org.apache.james.mailbox.model.UidValidity;
+import org.apache.james.mailbox.model.ParsedAttachment;
 import org.apache.james.mailbox.store.mail.AttachmentMapper;
 import org.apache.james.mailbox.store.mail.MessageMapper;
 import org.apache.james.mailbox.store.mail.model.impl.PropertyBuilder;
@@ -80,40 +81,35 @@ public abstract class MessageWithAttachmentMapperTest {
         this.attachmentMapper = mapperProvider.createAttachmentMapper();
 
         attachmentsMailbox = createMailbox(MailboxPath.forUser(Username.of("benwa"), "Attachments"));
-
-        Attachment attachment = Attachment.builder()
-                .attachmentId(AttachmentId.from("123"))
-                .bytes("attachment".getBytes())
-                .type("content")
-                .build();
-        Attachment attachment2 = Attachment.builder()
-                .attachmentId(AttachmentId.from("456"))
-                .bytes("attachment2".getBytes())
-                .type("content")
-                .build();
-        messageWith1Attachment = createMessage(attachmentsMailbox, mapperProvider.generateMessageId(), "Subject: Test7 \n\nBody7\n.\n", BODY_START, new PropertyBuilder(), 
-                ImmutableList.of(MessageAttachment.builder()
-                        .attachment(attachment)
-                        .cid(Cid.from("cid"))
-                        .isInline(true)
-                        .build()));
-        messageWith2Attachments = createMessage(attachmentsMailbox, mapperProvider.generateMessageId(), "Subject: Test8 \n\nBody8\n.\n", BODY_START, new PropertyBuilder(),
-                ImmutableList.of(
-                        MessageAttachment.builder()
-                            .attachment(attachment)
-                            .cid(Cid.from("cid"))
-                            .isInline(true)
-                            .build(),
-                        MessageAttachment.builder()
-                            .attachment(attachment2)
-                            .cid(Cid.from("cid2"))
-                            .isInline(false)
-                            .build()));
+        ParsedAttachment attachment1 = ParsedAttachment.builder()
+            .contentType("content")
+            .content(new ByteArrayInputStream("attachment".getBytes(StandardCharsets.UTF_8)))
+            .noName()
+            .cid(Cid.from("cid"))
+            .inline();
+        ParsedAttachment attachment2 = ParsedAttachment.builder()
+            .contentType("content")
+            .content(new ByteArrayInputStream("attachment2".getBytes(StandardCharsets.UTF_8)))
+            .noName()
+            .cid(Cid.from("cid"))
+            .inline();
+        ParsedAttachment attachment3 = ParsedAttachment.builder()
+            .contentType("content")
+            .content(new ByteArrayInputStream("attachment3".getBytes(StandardCharsets.UTF_8)))
+            .noName()
+            .cid(Cid.from("cid"))
+            .inline(false);
+
+        MessageId messageId1 = mapperProvider.generateMessageId();
+        MessageId messageId2 = mapperProvider.generateMessageId();
+        List<MessageAttachment> message1Attachments = attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment1), messageId1);
+        List<MessageAttachment> message2Attachments = attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment2, attachment3), messageId2);
+
+        messageWith1Attachment = createMessage(attachmentsMailbox, messageId1, "Subject: Test7 \n\nBody7\n.\n", BODY_START, new PropertyBuilder(),
+                message1Attachments);
+        messageWith2Attachments = createMessage(attachmentsMailbox, messageId2, "Subject: Test8 \n\nBody8\n.\n", BODY_START, new PropertyBuilder(),
+                message2Attachments);
         messageWithoutAttachment = createMessage(attachmentsMailbox, mapperProvider.generateMessageId(), "Subject: Test1 \n\nBody1\n.\n", BODY_START, new PropertyBuilder());
-
-        attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment), messageWith1Attachment.getMessageId());
-        attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment), messageWith2Attachments.getMessageId());
-        attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment2), messageWith2Attachments.getMessageId());
     }
 
     @Test


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