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 ad...@apache.org on 2017/09/20 20:31:59 UTC

[4/5] james-project git commit: JAMES-2145 Referencing explicitly some attachment owners is needed for attachment uploads (and bypass MessageId Checks)

JAMES-2145 Referencing explicitly some attachment owners is needed for attachment uploads (and bypass MessageId Checks)


Project: http://git-wip-us.apache.org/repos/asf/james-project/repo
Commit: http://git-wip-us.apache.org/repos/asf/james-project/commit/80009996
Tree: http://git-wip-us.apache.org/repos/asf/james-project/tree/80009996
Diff: http://git-wip-us.apache.org/repos/asf/james-project/diff/80009996

Branch: refs/heads/master
Commit: 80009996eae5f7c5d14d397a37464d54eccc44a4
Parents: de5e7fc
Author: benwa <bt...@linagora.com>
Authored: Tue Sep 19 13:27:33 2017 +0700
Committer: Antoine Duprat <ad...@linagora.com>
Committed: Wed Sep 20 16:20:55 2017 +0200

----------------------------------------------------------------------
 .../CassandraMailboxSessionMapperFactory.java   | 16 ++--
 .../mail/CassandraAttachmentMapper.java         | 14 ++-
 .../mail/CassandraAttachmentOwnerDAO.java       | 95 ++++++++++++++++++++
 .../modules/CassandraAttachmentModule.java      | 13 ++-
 .../table/CassandraAttachmentOwnerTable.java    | 29 ++++++
 .../CassandraSubscriptionManagerTest.java       |  3 +
 ...estCassandraMailboxSessionMapperFactory.java |  2 +
 .../mail/CassandraAttachmentFallbackTest.java   |  3 +-
 .../mail/CassandraAttachmentOwnerDAOTest.java   | 75 ++++++++++++++++
 .../inmemory/mail/InMemoryAttachmentMapper.java | 14 ++-
 .../mailbox/store/StoreAttachmentManager.java   | 22 ++++-
 .../mailbox/store/mail/AttachmentMapper.java    |  4 +-
 .../store/StoreAttachmentManagerTest.java       |  1 +
 .../store/mail/model/AttachmentMapperTest.java  | 74 +++++++++++++--
 .../model/MessageWithAttachmentMapperTest.java  |  5 +-
 15 files changed, 343 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMailboxSessionMapperFactory.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMailboxSessionMapperFactory.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMailboxSessionMapperFactory.java
index afffe6d..6806e42 100644
--- a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMailboxSessionMapperFactory.java
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraMailboxSessionMapperFactory.java
@@ -30,6 +30,7 @@ import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAOV2;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentMapper;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentMessageIdDAO;
+import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentOwnerDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraBlobsDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraDeletedMessageDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraFirstUnseenDAO;
@@ -83,9 +84,10 @@ public class CassandraMailboxSessionMapperFactory extends MailboxSessionMapperFa
     private final CassandraAttachmentDAOV2 attachmentDAOV2;
     private final CassandraDeletedMessageDAO deletedMessageDAO;
     private final CassandraBlobsDAO blobsDAO;
-    private final CassandraAttachmentMessageIdDAO attachementMessageIdDAO;
-    private CassandraUtils cassandraUtils;
-    private CassandraConfiguration cassandraConfiguration;
+    private final CassandraAttachmentMessageIdDAO attachmentMessageIdDAO;
+    private final CassandraAttachmentOwnerDAO ownerDAO;
+    private final CassandraUtils cassandraUtils;
+    private final CassandraConfiguration cassandraConfiguration;
 
     @Inject
     public CassandraMailboxSessionMapperFactory(CassandraUidProvider uidProvider, CassandraModSeqProvider modSeqProvider, Session session,
@@ -94,7 +96,8 @@ public class CassandraMailboxSessionMapperFactory extends MailboxSessionMapperFa
                                                 CassandraMailboxCounterDAO mailboxCounterDAO, CassandraMailboxRecentsDAO mailboxRecentsDAO, CassandraMailboxDAO mailboxDAO,
                                                 CassandraMailboxPathDAO mailboxPathDAO, CassandraFirstUnseenDAO firstUnseenDAO, CassandraApplicableFlagDAO applicableFlagDAO,
                                                 CassandraAttachmentDAO attachmentDAO, CassandraAttachmentDAOV2 attachmentDAOV2, CassandraDeletedMessageDAO deletedMessageDAO,
-                                                CassandraBlobsDAO blobsDAO, CassandraAttachmentMessageIdDAO attachementMessageIdDAO, CassandraUtils cassandraUtils, CassandraConfiguration cassandraConfiguration) {
+                                                CassandraBlobsDAO blobsDAO, CassandraAttachmentMessageIdDAO attachmentMessageIdDAO,
+                                                CassandraAttachmentOwnerDAO ownerDAO, CassandraUtils cassandraUtils, CassandraConfiguration cassandraConfiguration) {
         this.uidProvider = uidProvider;
         this.modSeqProvider = modSeqProvider;
         this.session = session;
@@ -111,8 +114,9 @@ public class CassandraMailboxSessionMapperFactory extends MailboxSessionMapperFa
         this.deletedMessageDAO = deletedMessageDAO;
         this.applicableFlagDAO = applicableFlagDAO;
         this.blobsDAO = blobsDAO;
-        this.attachementMessageIdDAO = attachementMessageIdDAO;
+        this.attachmentMessageIdDAO = attachmentMessageIdDAO;
         this.cassandraUtils = cassandraUtils;
+        this.ownerDAO = ownerDAO;
         this.cassandraConfiguration = cassandraConfiguration;
         this.indexTableHandler = new CassandraIndexTableHandler(
             mailboxRecentsDAO,
@@ -156,7 +160,7 @@ public class CassandraMailboxSessionMapperFactory extends MailboxSessionMapperFa
 
     @Override
     public CassandraAttachmentMapper createAttachmentMapper(MailboxSession mailboxSession) {
-        return new CassandraAttachmentMapper(attachmentDAO, attachmentDAOV2, blobsDAO, attachementMessageIdDAO);
+        return new CassandraAttachmentMapper(attachmentDAO, attachmentDAOV2, blobsDAO, attachmentMessageIdDAO, ownerDAO);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentMapper.java
----------------------------------------------------------------------
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 1269135..a0d2749 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
@@ -51,13 +51,15 @@ public class CassandraAttachmentMapper implements AttachmentMapper {
     private final CassandraAttachmentDAOV2 attachmentDAOV2;
     private final CassandraBlobsDAO blobsDAO;
     private final CassandraAttachmentMessageIdDAO attachmentMessageIdDAO;
+    private final CassandraAttachmentOwnerDAO ownerDAO;
 
     @Inject
-    public CassandraAttachmentMapper(CassandraAttachmentDAO attachmentDAO, CassandraAttachmentDAOV2 attachmentDAOV2, CassandraBlobsDAO blobsDAO, CassandraAttachmentMessageIdDAO attachmentMessageIdDAO) {
+    public CassandraAttachmentMapper(CassandraAttachmentDAO attachmentDAO, CassandraAttachmentDAOV2 attachmentDAOV2, CassandraBlobsDAO blobsDAO, CassandraAttachmentMessageIdDAO attachmentMessageIdDAO, final CassandraAttachmentOwnerDAO ownerDAO) {
         this.attachmentDAO = attachmentDAO;
         this.attachmentDAOV2 = attachmentDAOV2;
         this.blobsDAO = blobsDAO;
         this.attachmentMessageIdDAO = attachmentMessageIdDAO;
+        this.ownerDAO = ownerDAO;
     }
 
     @Override
@@ -120,8 +122,9 @@ public class CassandraAttachmentMapper implements AttachmentMapper {
     }
 
     @Override
-    public void storeAttachment(Attachment attachment) throws MailboxException {
-        blobsDAO.save(attachment.getBytes())
+    public void storeAttachmentForOwner(Attachment attachment, String owner) throws MailboxException {
+        ownerDAO.addOwner(attachment.getAttachmentId(), owner)
+            .thenCompose(any -> blobsDAO.save(attachment.getBytes()))
             .thenApply(blobId -> CassandraAttachmentDAOV2.from(attachment, blobId))
             .thenCompose(attachmentDAOV2::storeAttachment)
             .join();
@@ -141,6 +144,11 @@ public class CassandraAttachmentMapper implements AttachmentMapper {
             .join();
     }
 
+    @Override
+    public Collection<String> getOwners(AttachmentId attachmentId) throws MailboxException {
+        return ownerDAO.retrieveOwners(attachmentId).join();
+    }
+
     public CompletableFuture<Void> storeAttachmentAsync(Attachment attachment, MessageId ownerMessageId) {
         return blobsDAO.save(attachment.getBytes())
             .thenApply(blobId -> CassandraAttachmentDAOV2.from(attachment, blobId))

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAO.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAO.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAO.java
new file mode 100644
index 0000000..fe2d361
--- /dev/null
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAO.java
@@ -0,0 +1,95 @@
+/****************************************************************
+ * 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.cassandra.mail;
+
+import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.insertInto;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.select;
+import static org.apache.james.mailbox.cassandra.table.CassandraAttachmentOwnerTable.FIELDS;
+import static org.apache.james.mailbox.cassandra.table.CassandraAttachmentOwnerTable.ID;
+import static org.apache.james.mailbox.cassandra.table.CassandraAttachmentOwnerTable.OWNER;
+import static org.apache.james.mailbox.cassandra.table.CassandraAttachmentOwnerTable.TABLE_NAME;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Stream;
+
+import javax.inject.Inject;
+
+import org.apache.james.backends.cassandra.utils.CassandraAsyncExecutor;
+import org.apache.james.backends.cassandra.utils.CassandraUtils;
+import org.apache.james.mailbox.model.AttachmentId;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.github.steveash.guavate.Guavate;
+import com.google.common.collect.ImmutableList;
+
+public class CassandraAttachmentOwnerDAO {
+
+    private final CassandraAsyncExecutor executor;
+    private final PreparedStatement addStatement;
+    private final CassandraUtils cassandraUtils;
+    private final PreparedStatement selectStatement;
+
+    @Inject
+    public CassandraAttachmentOwnerDAO(Session session, CassandraUtils cassandraUtils) {
+        this.executor = new CassandraAsyncExecutor(session);
+
+        this.selectStatement = prepareSelect(session);
+        this.addStatement = prepareAdd(session);
+        this.cassandraUtils = cassandraUtils;
+    }
+
+    private PreparedStatement prepareAdd(Session session) {
+        return session.prepare(
+            insertInto(TABLE_NAME)
+                .value(ID, bindMarker(ID))
+                .value(OWNER, bindMarker(OWNER)));
+    }
+
+    private PreparedStatement prepareSelect(Session session) {
+        return session.prepare(
+            select(FIELDS)
+                .from(TABLE_NAME)
+                .where(eq(ID, bindMarker(ID))));
+    }
+
+    public CompletableFuture<Void> addOwner(AttachmentId attachmentId, String owner) {
+        return executor.executeVoid(
+            addStatement.bind()
+                .setUUID(ID, attachmentId.asUUID())
+                .setString(OWNER, owner));
+    }
+
+    public CompletableFuture<Collection<String>> retrieveOwners(AttachmentId attachmentId) {
+        return executor.execute(
+            selectStatement.bind()
+                .setUUID(ID, attachmentId.asUUID()))
+            .thenApply(cassandraUtils::convertToStream)
+            .thenApply(this::toOwners);
+    }
+
+    private ImmutableList<String> toOwners(Stream<Row> stream) {
+        return stream.map(row -> row.getString(OWNER)).collect(Guavate.toImmutableList());
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/modules/CassandraAttachmentModule.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/modules/CassandraAttachmentModule.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/modules/CassandraAttachmentModule.java
index dd2bc46..ffaa49a 100644
--- a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/modules/CassandraAttachmentModule.java
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/modules/CassandraAttachmentModule.java
@@ -31,6 +31,7 @@ import org.apache.james.backends.cassandra.components.CassandraTable;
 import org.apache.james.backends.cassandra.components.CassandraType;
 import org.apache.james.backends.cassandra.utils.CassandraConstants;
 import org.apache.james.mailbox.cassandra.table.CassandraAttachmentMessageIdTable;
+import org.apache.james.mailbox.cassandra.table.CassandraAttachmentOwnerTable;
 import org.apache.james.mailbox.cassandra.table.CassandraAttachmentTable;
 import org.apache.james.mailbox.cassandra.table.CassandraAttachmentV2Table;
 
@@ -77,7 +78,17 @@ public class CassandraAttachmentModule implements CassandraModule {
                     .compactionOptions(SchemaBuilder.leveledStrategy())
                     .caching(SchemaBuilder.KeyCaching.ALL,
                         SchemaBuilder.rows(CassandraConstants.DEFAULT_CACHED_ROW_PER_PARTITION))
-                    .comment("Holds ids of messages owning the attachment")));
+                    .comment("Holds ids of messages owning the attachment")),
+            new CassandraTable(CassandraAttachmentOwnerTable.TABLE_NAME,
+                SchemaBuilder.createTable(CassandraAttachmentOwnerTable.TABLE_NAME)
+                    .ifNotExists()
+                    .addPartitionKey(CassandraAttachmentOwnerTable.ID, uuid())
+                    .addClusteringColumn(CassandraAttachmentOwnerTable.OWNER, text())
+                    .withOptions()
+                    .compactionOptions(SchemaBuilder.leveledStrategy())
+                    .caching(SchemaBuilder.KeyCaching.ALL,
+                        SchemaBuilder.rows(CassandraConstants.DEFAULT_CACHED_ROW_PER_PARTITION))
+                    .comment("Holds explicit owners of some attachments")));
         types = ImmutableList.of();
     }
 

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/table/CassandraAttachmentOwnerTable.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/table/CassandraAttachmentOwnerTable.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/table/CassandraAttachmentOwnerTable.java
new file mode 100644
index 0000000..ff4b1a0
--- /dev/null
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/table/CassandraAttachmentOwnerTable.java
@@ -0,0 +1,29 @@
+/****************************************************************
+ * 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.cassandra.table;
+
+public interface CassandraAttachmentOwnerTable {
+
+    String TABLE_NAME = "attachmentOwners";
+    String ID = "id";
+    String OWNER = "owner";
+    String[] FIELDS = { ID, OWNER };
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/CassandraSubscriptionManagerTest.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/CassandraSubscriptionManagerTest.java b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/CassandraSubscriptionManagerTest.java
index 53345db..bcf3e6b 100644
--- a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/CassandraSubscriptionManagerTest.java
+++ b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/CassandraSubscriptionManagerTest.java
@@ -30,6 +30,7 @@ import org.apache.james.mailbox.cassandra.mail.CassandraApplicableFlagDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAOV2;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentMessageIdDAO;
+import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentOwnerDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraBlobsDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraDeletedMessageDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraFirstUnseenDAO;
@@ -93,6 +94,7 @@ public class CassandraSubscriptionManagerTest extends AbstractSubscriptionManage
         CassandraAttachmentDAOV2 attachmentDAOV2 = null;
         CassandraBlobsDAO cassandraBlobsDAO = null;
         CassandraAttachmentMessageIdDAO attachmentMessageIdDAO = null;
+        CassandraAttachmentOwnerDAO ownerDAO = null;
         return new CassandraSubscriptionManager(
             new CassandraMailboxSessionMapperFactory(
                 new CassandraUidProvider(cassandra.getConf()),
@@ -112,6 +114,7 @@ public class CassandraSubscriptionManagerTest extends AbstractSubscriptionManage
                 deletedMessageDAO,
                 cassandraBlobsDAO,
                 attachmentMessageIdDAO,
+                ownerDAO,
                 CassandraUtils.WITH_DEFAULT_CONFIGURATION,
                 CassandraConfiguration.DEFAULT_CONFIGURATION));
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/TestCassandraMailboxSessionMapperFactory.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/TestCassandraMailboxSessionMapperFactory.java b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/TestCassandraMailboxSessionMapperFactory.java
index 6f9d9e2..d5e450c 100644
--- a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/TestCassandraMailboxSessionMapperFactory.java
+++ b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/TestCassandraMailboxSessionMapperFactory.java
@@ -27,6 +27,7 @@ import org.apache.james.mailbox.cassandra.mail.CassandraApplicableFlagDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentDAOV2;
 import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentMessageIdDAO;
+import org.apache.james.mailbox.cassandra.mail.CassandraAttachmentOwnerDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraBlobsDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraDeletedMessageDAO;
 import org.apache.james.mailbox.cassandra.mail.CassandraFirstUnseenDAO;
@@ -64,6 +65,7 @@ public class TestCassandraMailboxSessionMapperFactory {
             new CassandraDeletedMessageDAO(session),
             cassandraBlobsDAO,
             new CassandraAttachmentMessageIdDAO(session, factory, CassandraUtils.WITH_DEFAULT_CONFIGURATION),
+            new CassandraAttachmentOwnerDAO(session, CassandraUtils.WITH_DEFAULT_CONFIGURATION),
             CassandraUtils.WITH_DEFAULT_CONFIGURATION,
             CassandraConfiguration.DEFAULT_CONFIGURATION);
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentFallbackTest.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentFallbackTest.java b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentFallbackTest.java
index d9ac703..6ac6409 100644
--- a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentFallbackTest.java
+++ b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentFallbackTest.java
@@ -74,7 +74,8 @@ public class CassandraAttachmentFallbackTest {
             CassandraConfiguration.DEFAULT_CONFIGURATION);
         blobsDAO = new CassandraBlobsDAO(cassandra.getConf());
         attachmentMessageIdDAO = new CassandraAttachmentMessageIdDAO(cassandra.getConf(), new CassandraMessageId.Factory(), CassandraUtils.WITH_DEFAULT_CONFIGURATION);
-        attachmentMapper = new CassandraAttachmentMapper(attachmentDAO, attachmentDAOV2, blobsDAO, attachmentMessageIdDAO);
+        CassandraAttachmentOwnerDAO ownerDAO = new CassandraAttachmentOwnerDAO(cassandra.getConf(), CassandraUtils.WITH_DEFAULT_CONFIGURATION);
+        attachmentMapper = new CassandraAttachmentMapper(attachmentDAO, attachmentDAOV2, blobsDAO, attachmentMessageIdDAO, ownerDAO);
     }
 
     @After

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAOTest.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAOTest.java b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAOTest.java
new file mode 100644
index 0000000..8082f5b
--- /dev/null
+++ b/mailbox/cassandra/src/test/java/org/apache/james/mailbox/cassandra/mail/CassandraAttachmentOwnerDAOTest.java
@@ -0,0 +1,75 @@
+/****************************************************************
+ * 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.cassandra.mail;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import org.apache.james.backends.cassandra.CassandraCluster;
+import org.apache.james.backends.cassandra.DockerCassandraRule;
+import org.apache.james.backends.cassandra.utils.CassandraUtils;
+import org.apache.james.mailbox.cassandra.modules.CassandraAttachmentModule;
+import org.apache.james.mailbox.model.AttachmentId;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+public class CassandraAttachmentOwnerDAOTest {
+    public static final AttachmentId ATTACHMENT_ID = AttachmentId.from("id1");
+    public static final String OWNER_1 = "owner1";
+    public static final String OWNER_2 = "owner2";
+
+    @ClassRule
+    public static DockerCassandraRule cassandraServer = new DockerCassandraRule();
+
+    private CassandraCluster cassandra;
+
+    private CassandraAttachmentOwnerDAO testee;
+
+    @Before
+    public void setUp() throws Exception {
+        cassandra = CassandraCluster.create(new CassandraAttachmentModule(),
+            cassandraServer.getIp(), cassandraServer.getBindingPort());
+        testee = new CassandraAttachmentOwnerDAO(cassandra.getConf(),
+            CassandraUtils.WITH_DEFAULT_CONFIGURATION);
+    }
+
+    @Test
+    public void retrieveOwnersShouldReturnEmptyByDefault() {
+        assertThat(testee.retrieveOwners(ATTACHMENT_ID).join())
+            .isEmpty();
+    }
+
+    @Test
+    public void retrieveOwnersShouldReturnAddedOwner() {
+        testee.addOwner(ATTACHMENT_ID, OWNER_1).join();
+
+        assertThat(testee.retrieveOwners(ATTACHMENT_ID).join())
+            .containsOnly(OWNER_1);
+    }
+
+    @Test
+    public void retrieveOwnersShouldReturnAddedOwners() {
+        testee.addOwner(ATTACHMENT_ID, OWNER_1).join();
+        testee.addOwner(ATTACHMENT_ID, OWNER_2).join();
+
+        assertThat(testee.retrieveOwners(ATTACHMENT_ID).join())
+            .containsOnly(OWNER_1, OWNER_2);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/mail/InMemoryAttachmentMapper.java
----------------------------------------------------------------------
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 cca1167..4ba0691 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
@@ -42,10 +42,12 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     private static final int INITIAL_SIZE = 128;
     private final Map<AttachmentId, Attachment> attachmentsById;
     private final Multimap<AttachmentId, MessageId> messageIdsByAttachmentId;
+    private final Multimap<AttachmentId, String> ownersByAttachmentId;
 
     public InMemoryAttachmentMapper() {
         attachmentsById = new ConcurrentHashMap<>(INITIAL_SIZE);
         messageIdsByAttachmentId = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+        ownersByAttachmentId = Multimaps.synchronizedSetMultimap(HashMultimap.create());
     }
 
     @Override
@@ -60,7 +62,7 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     @Override
     public List<Attachment> getAttachments(Collection<AttachmentId> attachmentIds) {
         Preconditions.checkArgument(attachmentIds != null);
-        Builder<Attachment> builder = ImmutableList.<Attachment> builder();
+        Builder<Attachment> builder = ImmutableList.builder();
         for (AttachmentId attachmentId : attachmentIds) {
             if (attachmentsById.containsKey(attachmentId)) {
                 builder.add(attachmentsById.get(attachmentId));
@@ -70,8 +72,9 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     }
 
     @Override
-    public void storeAttachment(Attachment attachment) throws MailboxException {
+    public void storeAttachmentForOwner(Attachment attachment, String owner) throws MailboxException {
         attachmentsById.put(attachment.getAttachmentId(), attachment);
+        ownersByAttachmentId.put(attachment.getAttachmentId(), owner);
     }
 
     @Override
@@ -87,7 +90,7 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     @Override
     public void storeAttachmentsForMessage(Collection<Attachment> attachments, MessageId ownerMessageId) throws MailboxException {
         for (Attachment attachment: attachments) {
-            storeAttachment(attachment);
+            attachmentsById.put(attachment.getAttachmentId(), attachment);
             messageIdsByAttachmentId.put(attachment.getAttachmentId(), ownerMessageId);
         }
     }
@@ -96,4 +99,9 @@ public class InMemoryAttachmentMapper implements AttachmentMapper {
     public Collection<MessageId> getRelatedMessageIds(AttachmentId attachmentId) throws MailboxException {
         return messageIdsByAttachmentId.get(attachmentId);
     }
+
+    @Override
+    public Collection<String> getOwners(final AttachmentId attachmentId) throws MailboxException {
+        return ownersByAttachmentId.get(attachmentId);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreAttachmentManager.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreAttachmentManager.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreAttachmentManager.java
index 87de834..7b6ce8a 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreAttachmentManager.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/StoreAttachmentManager.java
@@ -70,7 +70,8 @@ public class StoreAttachmentManager implements AttachmentManager {
 
     @Override
     public void storeAttachment(Attachment attachment, MailboxSession mailboxSession) throws MailboxException {
-        attachmentMapperFactory.getAttachmentMapper(mailboxSession).storeAttachment(attachment);
+        attachmentMapperFactory.getAttachmentMapper(mailboxSession)
+            .storeAttachmentForOwner(attachment, mailboxSession.getUser().getUserName());
     }
 
     @Override
@@ -80,15 +81,28 @@ public class StoreAttachmentManager implements AttachmentManager {
 
     private boolean userHasAccessToAttachment(AttachmentId attachmentId, MailboxSession mailboxSession) {
         try {
-            return !messageIdManager
-                .accessibleMessages(getRelatedMessageIds(attachmentId, mailboxSession), mailboxSession)
-                .isEmpty();
+            return isExplicitlyAOwner(attachmentId, mailboxSession)
+                || isReferencedInUserMessages(attachmentId, mailboxSession);
         } catch (MailboxException e) {
             LOGGER.warn("Error while checking attachment related accessible message ids", e);
             throw Throwables.propagate(e);
         }
     }
 
+    private boolean isReferencedInUserMessages(AttachmentId attachmentId, MailboxSession mailboxSession) throws MailboxException {
+        Collection<MessageId> relatedMessageIds = getRelatedMessageIds(attachmentId, mailboxSession);
+        return !messageIdManager
+            .accessibleMessages(relatedMessageIds, mailboxSession)
+            .isEmpty();
+    }
+
+    private boolean isExplicitlyAOwner(AttachmentId attachmentId, MailboxSession mailboxSession) throws MailboxException {
+        Collection<String> explicitOwners = attachmentMapperFactory.getAttachmentMapper(mailboxSession)
+            .getOwners(attachmentId);
+        return explicitOwners.stream()
+            .anyMatch(username -> mailboxSession.getUser().isSameUser(username));
+    }
+
     private Collection<MessageId> getRelatedMessageIds(AttachmentId attachmentId, MailboxSession mailboxSession) throws MailboxException {
         return attachmentMapperFactory.getAttachmentMapper(mailboxSession).getRelatedMessageIds(attachmentId);
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/store/src/main/java/org/apache/james/mailbox/store/mail/AttachmentMapper.java
----------------------------------------------------------------------
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 3bcb496..7973a20 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
@@ -34,9 +34,11 @@ public interface AttachmentMapper extends Mapper {
 
     List<Attachment> getAttachments(Collection<AttachmentId> attachmentIds);
 
-    void storeAttachment(Attachment attachment) throws MailboxException;
+    void storeAttachmentForOwner(Attachment attachment, String owner) throws MailboxException;
 
     void storeAttachmentsForMessage(Collection<Attachment> attachments, MessageId ownerMessageId) throws MailboxException;
 
     Collection<MessageId> getRelatedMessageIds(AttachmentId attachmentId) throws MailboxException;
+
+    Collection<String> getOwners(AttachmentId attachmentId) throws MailboxException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/store/src/test/java/org/apache/james/mailbox/store/StoreAttachmentManagerTest.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/test/java/org/apache/james/mailbox/store/StoreAttachmentManagerTest.java b/mailbox/store/src/test/java/org/apache/james/mailbox/store/StoreAttachmentManagerTest.java
index f205808..d753fca 100644
--- a/mailbox/store/src/test/java/org/apache/james/mailbox/store/StoreAttachmentManagerTest.java
+++ b/mailbox/store/src/test/java/org/apache/james/mailbox/store/StoreAttachmentManagerTest.java
@@ -71,6 +71,7 @@ public class StoreAttachmentManagerTest {
         MailboxSession mailboxSession = mock(MailboxSession.class);
         when(attachmentMapper.getAttachment(ATTACHMENT_ID)).thenReturn(ATTACHMENT);
         when(attachmentMapper.getRelatedMessageIds(ATTACHMENT_ID)).thenReturn(MESSAGE_IDS);
+        when(attachmentMapper.getOwners(ATTACHMENT_ID)).thenReturn(ImmutableList.of());
         when(messageIdManager.accessibleMessages(MESSAGE_IDS, mailboxSession)).thenReturn(ImmutableSet.of());
 
         assertThatThrownBy(() -> testee.getAttachment(ATTACHMENT_ID, mailboxSession))

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/AttachmentMapperTest.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/AttachmentMapperTest.java b/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/AttachmentMapperTest.java
index 9459ed9..bc411a9 100644
--- a/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/AttachmentMapperTest.java
+++ b/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/AttachmentMapperTest.java
@@ -40,9 +40,10 @@ import com.google.common.collect.ImmutableList;
 
 public abstract class AttachmentMapperTest {
     private static final AttachmentId UNKNOWN_ATTACHMENT_ID = AttachmentId.forPayloadAndType("unknown".getBytes(Charsets.UTF_8), "type");
+    public static final String OWNER = "owner";
+    public static final String ADDITIONAL_OWNER = "additionalOwner";
 
     private AttachmentMapper attachmentMapper;
-    private MapperProvider mapperProvider;
 
     @Rule
     public ExpectedException expected = ExpectedException.none();
@@ -52,7 +53,7 @@ public abstract class AttachmentMapperTest {
     protected abstract MessageId generateMessageId();
 
     public void setUp() throws MailboxException {
-        mapperProvider = createMapperProvider();
+        MapperProvider mapperProvider = createMapperProvider();
         Assume.assumeTrue(mapperProvider.getSupportedCapabilities().contains(MapperProvider.Capabilities.ATTACHMENT));
         this.attachmentMapper = mapperProvider.createAttachmentMapper();
     }
@@ -77,7 +78,7 @@ public abstract class AttachmentMapperTest {
                 .type("content")
                 .build();
         AttachmentId attachmentId = expected.getAttachmentId();
-        attachmentMapper.storeAttachment(expected);
+        attachmentMapper.storeAttachmentForOwner(expected, OWNER);
         //When
         Attachment attachment = attachmentMapper.getAttachment(attachmentId);
         //Then
@@ -127,14 +128,14 @@ public abstract class AttachmentMapperTest {
                 .type("content")
                 .build();
         AttachmentId attachmentId = expected.getAttachmentId();
-        attachmentMapper.storeAttachment(expected);
+        attachmentMapper.storeAttachmentForOwner(expected, OWNER);
 
         Attachment expected2 = Attachment.builder()
                 .bytes("payload2".getBytes(Charsets.UTF_8))
                 .type("content")
                 .build();
         AttachmentId attachmentId2 = expected2.getAttachmentId();
-        attachmentMapper.storeAttachment(expected2);
+        attachmentMapper.storeAttachmentForOwner(expected2, OWNER);
 
         //When
         List<Attachment> attachments = attachmentMapper.getAttachments(ImmutableList.of(attachmentId, attachmentId2));
@@ -157,7 +158,7 @@ public abstract class AttachmentMapperTest {
                 .type("content")
                 .build();
         AttachmentId attachmentId = attachment.getAttachmentId();
-        attachmentMapper.storeAttachment(attachment);
+        attachmentMapper.storeAttachmentForOwner(attachment, OWNER);
         
         //When
         Collection<MessageId> messageIds = attachmentMapper.getRelatedMessageIds(attachmentId);
@@ -264,4 +265,65 @@ public abstract class AttachmentMapperTest {
         //Then
         assertThat(messageIds).isEqualTo(messageIds2);
     }
+
+    @Test
+    public void getOwnersShouldBeRetrievedWhenExplicitlySpecified() throws Exception {
+        //Given
+        Attachment attachment = Attachment.builder()
+            .bytes("payload".getBytes(Charsets.UTF_8))
+            .type("content")
+            .build();
+
+        AttachmentId attachmentId = attachment.getAttachmentId();
+        attachmentMapper.storeAttachmentForOwner(attachment, OWNER);
+
+        //When
+        Collection<String> expectedOwners = ImmutableList.of(OWNER);
+        Collection<String> actualOwners = attachmentMapper.getOwners(attachmentId);
+        //Then
+        assertThat(actualOwners).containsOnlyElementsOf(expectedOwners);
+    }
+
+    @Test
+    public void getOwnersShouldReturnEmptyWhenMessageIdReferenced() throws Exception {
+        //Given
+        Attachment attachment = Attachment.builder()
+            .bytes("payload".getBytes(Charsets.UTF_8))
+            .type("content")
+            .build();
+
+        AttachmentId attachmentId = attachment.getAttachmentId();
+        attachmentMapper.storeAttachmentsForMessage(ImmutableList.of(attachment), generateMessageId());
+
+        //When
+        Collection<String> actualOwners = attachmentMapper.getOwners(attachmentId);
+        //Then
+        assertThat(actualOwners).isEmpty();
+    }
+
+    @Test
+    public void getOwnersShouldReturnAllOwners() throws Exception {
+        //Given
+        Attachment attachment = Attachment.builder()
+            .bytes("payload".getBytes(Charsets.UTF_8))
+            .type("content")
+            .build();
+
+        AttachmentId attachmentId = attachment.getAttachmentId();
+        attachmentMapper.storeAttachmentForOwner(attachment, OWNER);
+        attachmentMapper.storeAttachmentForOwner(attachment, ADDITIONAL_OWNER);
+
+        //When
+        Collection<String> expectedOwners = ImmutableList.of(OWNER, ADDITIONAL_OWNER);
+        Collection<String> actualOwners = attachmentMapper.getOwners(attachmentId);
+        //Then
+        assertThat(actualOwners).containsOnlyElementsOf(expectedOwners);
+    }
+
+    @Test
+    public void getOwnersShouldReturnEmptyWhenUnknownAttachmentId() throws Exception {
+        Collection<String> actualOwners = attachmentMapper.getOwners(AttachmentId.from("any"));
+
+        assertThat(actualOwners).isEmpty();
+    }
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/80009996/mailbox/store/src/test/java/org/apache/james/mailbox/store/mail/model/MessageWithAttachmentMapperTest.java
----------------------------------------------------------------------
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 3320e67..59e31c3 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
@@ -88,13 +88,11 @@ public abstract class MessageWithAttachmentMapperTest {
                 .bytes("attachment".getBytes())
                 .type("content")
                 .build();
-        attachmentMapper.storeAttachment(attachment);
         Attachment attachment2 = Attachment.builder()
                 .attachmentId(AttachmentId.from("456"))
                 .bytes("attachment2".getBytes())
                 .type("content")
                 .build();
-        attachmentMapper.storeAttachment(attachment2);
         messageWith1Attachment = createMessage(attachmentsMailbox, mapperProvider.generateMessageId(), "Subject: Test7 \n\nBody7\n.\n", BODY_START, new PropertyBuilder(), 
                 ImmutableList.of(MessageAttachment.builder()
                         .attachment(attachment)
@@ -115,6 +113,9 @@ public abstract class MessageWithAttachmentMapperTest {
                             .build()));
         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