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 ma...@apache.org on 2016/07/25 08:33:57 UTC

[6/8] james-project git commit: MAILBOX-273 implement cross-mailboxes search in ElasticSearch

MAILBOX-273 implement cross-mailboxes search in ElasticSearch


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

Branch: refs/heads/master
Commit: 085cf65e9dbb1dfe5b3fbe4a00c7d9a36ea334be
Parents: eff623d
Author: Matthieu Baechler <ma...@linagora.com>
Authored: Tue Jul 19 18:12:07 2016 +0200
Committer: Matthieu Baechler <ma...@linagora.com>
Committed: Mon Jul 25 10:13:30 2016 +0200

----------------------------------------------------------------------
 .../apache/james/mailbox/model/MailboxId.java   |   5 +
 .../model/MultimailboxesSearchQuery.java        |  72 ++++++++++++
 .../model/MultimailboxesSearchQueryTest.java    |  92 +++++++++++++++
 .../james/mailbox/cassandra/CassandraId.java    |   7 ++
 mailbox/elasticsearch/pom.xml                   |   5 +
 ...lasticSearchListeningMessageSearchIndex.java |  17 ++-
 .../elasticsearch/query/QueryConverter.java     |  26 +++--
 .../search/ElasticSearchSearcher.java           |  51 ++++----
 .../ElasticSearchIntegrationTest.java           |   3 +-
 .../lucene/search/LuceneMessageSearchIndex.java | 108 ++++++++++++-----
 .../LuceneMailboxMessageSearchIndexTest.java    |  34 +++++-
 .../search/LuceneMessageSearchIndexTest.java    |  15 ++-
 .../james/mailbox/inmemory/InMemoryId.java      |   8 ++
 .../search/SimpleMessageSearchIndexTest.java    |  25 ++++
 .../store/search/LazyMessageSearchIndex.java    |  10 ++
 .../store/search/MessageSearchIndex.java        |  15 ++-
 .../store/search/SimpleMessageSearchIndex.java  |  10 ++
 .../org/apache/james/mailbox/store/TestId.java  |   8 ++
 .../search/AbstractMessageSearchIndexTest.java  | 116 ++++++++++++++++---
 .../imap/scripts/UidSearchAtomsIndexer.test     |  37 +++++-
 .../host/ElasticSearchHostSystem.java           |   3 +-
 .../modules/mailbox/CassandraMailboxModule.java |   4 +
 22 files changed, 583 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/api/src/main/java/org/apache/james/mailbox/model/MailboxId.java
----------------------------------------------------------------------
diff --git a/mailbox/api/src/main/java/org/apache/james/mailbox/model/MailboxId.java b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MailboxId.java
index 7410b4e..3e263fc 100644
--- a/mailbox/api/src/main/java/org/apache/james/mailbox/model/MailboxId.java
+++ b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MailboxId.java
@@ -19,5 +19,10 @@
 package org.apache.james.mailbox.model;
 
 public interface MailboxId {
+    
+    interface Factory {
+        MailboxId fromString(String serialized);
+    }
+    
     String serialize();
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/api/src/main/java/org/apache/james/mailbox/model/MultimailboxesSearchQuery.java
----------------------------------------------------------------------
diff --git a/mailbox/api/src/main/java/org/apache/james/mailbox/model/MultimailboxesSearchQuery.java b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MultimailboxesSearchQuery.java
new file mode 100644
index 0000000..f979a79
--- /dev/null
+++ b/mailbox/api/src/main/java/org/apache/james/mailbox/model/MultimailboxesSearchQuery.java
@@ -0,0 +1,72 @@
+/****************************************************************
+ * 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.util.Arrays;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+
+public class MultimailboxesSearchQuery {
+
+    public static Builder from(SearchQuery searchQuery) {
+        return new Builder(searchQuery);
+    }
+    
+    public static class Builder {
+        
+        private final SearchQuery searchQuery;
+        private ImmutableSet.Builder<MailboxId> mailboxIds;
+
+        private Builder(SearchQuery searchQuery) {
+            Preconditions.checkNotNull(searchQuery);
+            this.searchQuery = searchQuery;
+            this.mailboxIds = ImmutableSet.builder();
+        }
+        
+        public Builder inMailboxes(MailboxId... mailboxIds) {
+            this.mailboxIds.addAll(Arrays.asList(mailboxIds));
+            return this;
+        }
+        
+        public MultimailboxesSearchQuery build() {
+            return new MultimailboxesSearchQuery(searchQuery, mailboxIds.build());
+        }
+        
+    }
+
+    private final SearchQuery searchQuery;
+    private final ImmutableSet<MailboxId> mailboxIds;
+
+    @VisibleForTesting
+    MultimailboxesSearchQuery(SearchQuery searchQuery, ImmutableSet<MailboxId> mailboxIds) {
+        this.searchQuery = searchQuery;
+        this.mailboxIds = mailboxIds;
+    }
+
+    public ImmutableSet<MailboxId> getMailboxIds() {
+        return mailboxIds;
+    }
+    
+    public SearchQuery getSearchQuery() {
+        return searchQuery;
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/api/src/test/java/org/apache/james/mailbox/model/MultimailboxesSearchQueryTest.java
----------------------------------------------------------------------
diff --git a/mailbox/api/src/test/java/org/apache/james/mailbox/model/MultimailboxesSearchQueryTest.java b/mailbox/api/src/test/java/org/apache/james/mailbox/model/MultimailboxesSearchQueryTest.java
new file mode 100644
index 0000000..92a77a2
--- /dev/null
+++ b/mailbox/api/src/test/java/org/apache/james/mailbox/model/MultimailboxesSearchQueryTest.java
@@ -0,0 +1,92 @@
+/****************************************************************
+ * 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 static org.assertj.core.api.Assertions.assertThat;
+
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableSet;
+
+public class MultimailboxesSearchQueryTest {
+
+    private static final SearchQuery EMPTY_QUERY = new SearchQuery();
+
+    @Test(expected=NullPointerException.class)
+    public void buildShouldThrowWhenQueryIsNull() {
+        MultimailboxesSearchQuery.from(null);
+    }
+
+    @Test
+    public void buildShouldBuildWhenNoMailboxes() {
+        MultimailboxesSearchQuery expected = new MultimailboxesSearchQuery(EMPTY_QUERY, ImmutableSet.<MailboxId>of());
+        MultimailboxesSearchQuery actual = MultimailboxesSearchQuery.from(EMPTY_QUERY).build();
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    public void buildShouldBuildWhenEmptyMailboxes() {
+        MultimailboxesSearchQuery expected = new MultimailboxesSearchQuery(EMPTY_QUERY, ImmutableSet.<MailboxId>of());
+        MultimailboxesSearchQuery actual = MultimailboxesSearchQuery.from(EMPTY_QUERY).inMailboxes().build();
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    public void buildShouldBuildWhenOneMailbox() {
+        MultimailboxesSearchQuery expected = new MultimailboxesSearchQuery(EMPTY_QUERY, ImmutableSet.<MailboxId>of(TestId.of(("id1"))));
+        MultimailboxesSearchQuery actual = MultimailboxesSearchQuery.from(EMPTY_QUERY).inMailboxes(TestId.of("id1")).build();
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    public void buildShouldBuildWhenTwoMailboxes() {
+        MultimailboxesSearchQuery expected = new MultimailboxesSearchQuery(EMPTY_QUERY, ImmutableSet.<MailboxId>of(TestId.of("id1"), TestId.of("id2")));
+        MultimailboxesSearchQuery actual = MultimailboxesSearchQuery.from(EMPTY_QUERY).inMailboxes(TestId.of("id1"), TestId.of("id2")).build();
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    private static class TestId implements MailboxId {
+        private final String id;
+
+        public TestId(String id) {
+            this.id = id;
+        }
+
+        public static TestId of(String id) {
+            return new TestId(id);
+        }
+        
+        @Override
+        public String serialize() {
+            return id;
+        }
+        
+        @Override
+        public boolean equals(Object other) {
+            return other instanceof TestId
+                    && id.equals(((TestId)other).id);
+        }
+        
+        @Override
+        public int hashCode() {
+            return id.hashCode();
+        };
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraId.java
----------------------------------------------------------------------
diff --git a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraId.java b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraId.java
index 76088d8..7553d6d 100644
--- a/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraId.java
+++ b/mailbox/cassandra/src/main/java/org/apache/james/mailbox/cassandra/CassandraId.java
@@ -26,6 +26,13 @@ import com.datastax.driver.core.utils.UUIDs;
 
 public class CassandraId implements MailboxId {
 
+    public static class Factory implements MailboxId.Factory {
+        @Override
+        public MailboxId fromString(String serialized) {
+            return of(UUID.fromString(serialized));
+        }
+    }
+    
     private final UUID id;
 
     public static CassandraId timeBased() {

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/mailbox/elasticsearch/pom.xml b/mailbox/elasticsearch/pom.xml
index 793a05d..864a95f 100644
--- a/mailbox/elasticsearch/pom.xml
+++ b/mailbox/elasticsearch/pom.xml
@@ -190,6 +190,11 @@
                     <artifactId>jackson-datatype-jdk8</artifactId>
                 </dependency>
                 <dependency>
+                    <groupId>com.github.steveash.guavate</groupId>
+                    <artifactId>guavate</artifactId>
+                    <version>1.0.0</version>
+                </dependency>
+                <dependency>
                     <groupId>com.google.guava</groupId>
                     <artifactId>guava</artifactId>
                 </dependency>

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/events/ElasticSearchListeningMessageSearchIndex.java
----------------------------------------------------------------------
diff --git a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/events/ElasticSearchListeningMessageSearchIndex.java b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/events/ElasticSearchListeningMessageSearchIndex.java
index ef053b5..b967e7c 100644
--- a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/events/ElasticSearchListeningMessageSearchIndex.java
+++ b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/events/ElasticSearchListeningMessageSearchIndex.java
@@ -20,8 +20,10 @@ package org.apache.james.mailbox.elasticsearch.events;
 
 import static org.elasticsearch.index.query.QueryBuilders.termQuery;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 
 import javax.inject.Inject;
@@ -32,6 +34,8 @@ import org.apache.james.mailbox.elasticsearch.json.JsonMessageConstants;
 import org.apache.james.mailbox.elasticsearch.json.MessageToElasticSearchJson;
 import org.apache.james.mailbox.elasticsearch.search.ElasticSearchSearcher;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.UpdatedFlags;
 import org.apache.james.mailbox.store.mail.MessageMapperFactory;
@@ -42,6 +46,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.collect.ImmutableList;
 
 public class ElasticSearchListeningMessageSearchIndex extends ListeningMessageSearchIndex {
 
@@ -68,7 +73,17 @@ public class ElasticSearchListeningMessageSearchIndex extends ListeningMessageSe
 
     @Override
     public Iterator<Long> search(MailboxSession session, Mailbox mailbox, SearchQuery searchQuery) throws MailboxException {
-        return searcher.search(mailbox, searchQuery);
+        MailboxId mailboxId = mailbox.getMailboxId();
+        return searcher
+                .search(ImmutableList.of(mailboxId), searchQuery)
+                .get(mailboxId)
+                .iterator();
+    }
+    
+    @Override
+    public Map<MailboxId, Collection<Long>> search(MailboxSession session, MultimailboxesSearchQuery searchQuery)
+            throws MailboxException {
+        return searcher.search(searchQuery.getMailboxIds(), searchQuery.getSearchQuery()).asMap();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/query/QueryConverter.java
----------------------------------------------------------------------
diff --git a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/query/QueryConverter.java b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/query/QueryConverter.java
index b0a5346..762522f 100644
--- a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/query/QueryConverter.java
+++ b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/query/QueryConverter.java
@@ -20,17 +20,21 @@
 package org.apache.james.mailbox.elasticsearch.query;
 
 import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
-import static org.elasticsearch.index.query.QueryBuilders.termQuery;
+import static org.elasticsearch.index.query.QueryBuilders.termsQuery;
 
+import java.util.Collection;
 import java.util.List;
-import java.util.stream.Stream;
 
 import javax.inject.Inject;
 
 import org.apache.james.mailbox.elasticsearch.json.JsonMessageConstants;
+import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.elasticsearch.index.query.QueryBuilder;
 
+import com.github.steveash.guavate.Guavate;
+import com.google.common.collect.ImmutableList;
+
 public class QueryConverter {
 
 
@@ -41,11 +45,9 @@ public class QueryConverter {
         this.criterionConverter = criterionConverter;
     }
 
-    public QueryBuilder from(SearchQuery searchQuery, String mailboxUUID) {
-        return Stream.of(generateQueryBuilder(searchQuery))
-            .map((rep) -> addMailboxFilters(rep, mailboxUUID))
-            .findAny()
-            .get();
+    public QueryBuilder from(SearchQuery searchQuery, Collection<MailboxId> mailboxIds) {
+        QueryBuilder queryBuilder = generateQueryBuilder(searchQuery);
+        return addMailboxFilters(queryBuilder, mailboxIds);
     }
 
     private QueryBuilder generateQueryBuilder(SearchQuery searchQuery) {
@@ -59,9 +61,15 @@ public class QueryConverter {
         }
     }
 
-    private QueryBuilder addMailboxFilters(QueryBuilder queryBuilder, String mailboxUUID) {
+    private QueryBuilder addMailboxFilters(QueryBuilder queryBuilder, Collection<MailboxId> mailboxIds) {
+        if (mailboxIds.isEmpty()) {
+            return queryBuilder;
+        }
+        ImmutableList<String> ids = mailboxIds.stream()
+                .map(MailboxId::serialize)
+                .collect(Guavate.toImmutableList());
         return boolQuery().must(queryBuilder)
-            .filter(termQuery(JsonMessageConstants.MAILBOX_ID, mailboxUUID));
+            .filter(termsQuery(JsonMessageConstants.MAILBOX_ID, ids));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/search/ElasticSearchSearcher.java
----------------------------------------------------------------------
diff --git a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/search/ElasticSearchSearcher.java b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/search/ElasticSearchSearcher.java
index 1042ec8..8a9a1d8 100644
--- a/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/search/ElasticSearchSearcher.java
+++ b/mailbox/elasticsearch/src/main/java/org/apache/james/mailbox/elasticsearch/search/ElasticSearchSearcher.java
@@ -19,28 +19,34 @@
 
 package org.apache.james.mailbox.elasticsearch.search;
 
-import java.util.Iterator;
+import java.util.Collection;
 import java.util.Optional;
 import java.util.stream.Stream;
 import java.util.stream.StreamSupport;
 
 import javax.inject.Inject;
 
+import org.apache.commons.lang3.tuple.Pair;
 import org.apache.james.mailbox.elasticsearch.ElasticSearchIndexer;
 import org.apache.james.mailbox.elasticsearch.json.JsonMessageConstants;
 import org.apache.james.mailbox.elasticsearch.query.QueryConverter;
 import org.apache.james.mailbox.elasticsearch.query.SortConverter;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.mailbox.model.MailboxId.Factory;
 import org.apache.james.mailbox.model.SearchQuery;
-import org.apache.james.mailbox.store.mail.model.Mailbox;
 import org.elasticsearch.action.search.SearchRequestBuilder;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.common.unit.TimeValue;
 import org.elasticsearch.search.SearchHit;
+import org.elasticsearch.search.SearchHitField;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import com.github.steveash.guavate.Guavate;
+import com.google.common.collect.Multimap;
+
 public class ElasticSearchSearcher {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(ElasticSearchSearcher.class);
@@ -50,50 +56,55 @@ public class ElasticSearchSearcher {
     private final Client client;
     private final QueryConverter queryConverter;
     private final int size;
+    private final Factory mailboxIdFactory;
 
     @Inject
-    public ElasticSearchSearcher(Client client, QueryConverter queryConverter) {
-        this(client, queryConverter, DEFAULT_SIZE);
+    public ElasticSearchSearcher(Client client, QueryConverter queryConverter, MailboxId.Factory mailboxIdFactory) {
+        this(client, queryConverter, DEFAULT_SIZE, mailboxIdFactory);
     }
 
-    public ElasticSearchSearcher(Client client, QueryConverter queryConverter, int size) {
+    public ElasticSearchSearcher(Client client, QueryConverter queryConverter, int size, MailboxId.Factory mailboxIdFactory) {
         this.client = client;
         this.queryConverter = queryConverter;
         this.size = size;
+        this.mailboxIdFactory = mailboxIdFactory;
     }
-
-    public Iterator<Long> search(Mailbox mailbox, SearchQuery searchQuery) throws MailboxException {
-        return new ScrollIterable(client, getSearchRequestBuilder(client, mailbox, searchQuery)).stream()
+    
+    public Multimap<MailboxId, Long> search(Collection<MailboxId> mailboxIds, SearchQuery searchQuery) throws MailboxException {
+        return new ScrollIterable(client, getSearchRequestBuilder(client, mailboxIds, searchQuery)).stream()
             .flatMap(this::transformResponseToUidStream)
-            .iterator();
+            .collect(Guavate.toImmutableListMultimap(Pair::getLeft, Pair::getRight));
     }
-
-    private SearchRequestBuilder getSearchRequestBuilder(Client client, Mailbox mailbox, SearchQuery searchQuery) {
+    
+    private SearchRequestBuilder getSearchRequestBuilder(Client client, Collection<MailboxId> mailboxIds, SearchQuery searchQuery) {
         return searchQuery.getSorts()
             .stream()
             .reduce(
                 client.prepareSearch(ElasticSearchIndexer.MAILBOX_INDEX)
                     .setTypes(ElasticSearchIndexer.MESSAGE_TYPE)
                     .setScroll(TIMEOUT)
-                    .setFetchSource(JsonMessageConstants.ID, "")
-                    .setQuery(queryConverter.from(searchQuery, mailbox.getMailboxId().serialize()))
+                    .addFields(JsonMessageConstants.ID, JsonMessageConstants.MAILBOX_ID)
+                    .setQuery(queryConverter.from(searchQuery, mailboxIds))
                     .setSize(size),
                 (searchBuilder, sort) -> searchBuilder.addSort(SortConverter.convertSort(sort)),
                 (partialResult1, partialResult2) -> partialResult1);
     }
 
-    private Stream<Long> transformResponseToUidStream(SearchResponse searchResponse) {
+    private Stream<Pair<MailboxId, Long>> transformResponseToUidStream(SearchResponse searchResponse) {
         return StreamSupport.stream(searchResponse.getHits().spliterator(), false)
-            .map(this::extractUidFromHit)
+            .map(this::extractContentFromHit)
             .filter(Optional::isPresent)
             .map(Optional::get);
     }
 
-    private Optional<Long> extractUidFromHit(SearchHit hit) {
-        try {
-            return Optional.of(((Number) hit.getSource().get(JsonMessageConstants.ID)).longValue());
-        } catch (Exception exception) {
-            LOGGER.warn("Can not extract UID for search result " + hit.getId(), exception);
+    private Optional<Pair<MailboxId, Long>> extractContentFromHit(SearchHit hit) {
+        SearchHitField mailboxId = hit.field(JsonMessageConstants.MAILBOX_ID);
+        SearchHitField uid = hit.field(JsonMessageConstants.ID);
+        if (mailboxId != null && uid != null) {
+            Number uidAsNumber = uid.getValue();
+            return Optional.of(Pair.of(mailboxIdFactory.fromString(mailboxId.getValue()), uidAsNumber.longValue()));
+        } else {
+            LOGGER.warn("Can not extract UID and/or MailboxId for search result " + hit.getId());
             return Optional.empty();
         }
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/elasticsearch/src/test/java/org/apache/james/mailbox/elasticsearch/ElasticSearchIntegrationTest.java
----------------------------------------------------------------------
diff --git a/mailbox/elasticsearch/src/test/java/org/apache/james/mailbox/elasticsearch/ElasticSearchIntegrationTest.java b/mailbox/elasticsearch/src/test/java/org/apache/james/mailbox/elasticsearch/ElasticSearchIntegrationTest.java
index 8467afd..5d55b63 100644
--- a/mailbox/elasticsearch/src/test/java/org/apache/james/mailbox/elasticsearch/ElasticSearchIntegrationTest.java
+++ b/mailbox/elasticsearch/src/test/java/org/apache/james/mailbox/elasticsearch/ElasticSearchIntegrationTest.java
@@ -30,6 +30,7 @@ import org.apache.james.mailbox.elasticsearch.query.CriterionConverter;
 import org.apache.james.mailbox.elasticsearch.query.QueryConverter;
 import org.apache.james.mailbox.elasticsearch.search.ElasticSearchSearcher;
 import org.apache.james.mailbox.elasticsearch.utils.TestingClientProvider;
+import org.apache.james.mailbox.inmemory.InMemoryId;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxManager;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxSessionMapperFactory;
 import org.apache.james.mailbox.store.JVMMailboxPathLocker;
@@ -67,7 +68,7 @@ public class ElasticSearchIntegrationTest extends AbstractMessageSearchIndexTest
         MailboxSessionMapperFactory mapperFactory = new InMemoryMailboxSessionMapperFactory();
         messageSearchIndex = new ElasticSearchListeningMessageSearchIndex(mapperFactory,
             new ElasticSearchIndexer(client, new DeleteByQueryPerformer(client, Executors.newSingleThreadExecutor(), BATCH_SIZE)),
-            new ElasticSearchSearcher(client, new QueryConverter(new CriterionConverter()), SEARCH_SIZE),
+            new ElasticSearchSearcher(client, new QueryConverter(new CriterionConverter()), SEARCH_SIZE, new InMemoryId.Factory()),
             new MessageToElasticSearchJson(new DefaultTextExtractor(), ZoneId.of("Europe/Paris")));
         storeMailboxManager = new InMemoryMailboxManager(
             mapperFactory,

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/lucene/src/main/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndex.java
----------------------------------------------------------------------
diff --git a/mailbox/lucene/src/main/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndex.java b/mailbox/lucene/src/main/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndex.java
index c9585b5..13ce4ac 100644
--- a/mailbox/lucene/src/main/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndex.java
+++ b/mailbox/lucene/src/main/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndex.java
@@ -30,9 +30,9 @@ import java.util.Collection;
 import java.util.Date;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Locale;
+import java.util.Map;
 import java.util.Set;
 import java.util.TimeZone;
 
@@ -42,7 +42,10 @@ import javax.mail.Flags.Flag;
 import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.exception.MailboxException;
 import org.apache.james.mailbox.exception.UnsupportedSearchException;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.mailbox.model.MailboxId.Factory;
 import org.apache.james.mailbox.model.MessageRange;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.SearchQuery.AllCriterion;
 import org.apache.james.mailbox.model.SearchQuery.ContainsOperator;
@@ -94,6 +97,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
 import org.apache.lucene.search.NumericRangeQuery;
 import org.apache.lucene.search.PrefixQuery;
 import org.apache.lucene.search.Query;
@@ -109,6 +113,10 @@ import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.Version;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Multimap;
+
 /**
  * Lucene based {@link ListeningMessageSearchIndex} which offers message searching via a Lucene index
  * 
@@ -296,12 +304,6 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
     private final static String MEDIA_TYPE_MESSAGE = "message"; 
     private final static String DEFAULT_ENCODING = "US-ASCII";
     
-    private final IndexWriter writer;
-    
-    private int maxQueryResults = DEFAULT_MAX_QUERY_RESULTS;
-
-    private boolean suffixMatch = false;
-    
     private final static SortField UID_SORT = new SortField(UID_FIELD, SortField.LONG);
     private final static SortField UID_SORT_REVERSE = new SortField(UID_FIELD, SortField.LONG, true);
 
@@ -332,21 +334,30 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
 
     private final static SortField FIRST_FROM_MAILBOX_DISPLAY_SORT = new SortField(FIRST_FROM_MAILBOX_DISPLAY_FIELD, SortField.STRING);
     private final static SortField FIRST_FROM_MAILBOX_DISPLAY_SORT_REVERSE = new SortField(FIRST_FROM_MAILBOX_DISPLAY_FIELD, SortField.STRING, true);
+    
+    private final Factory mailboxIdFactory;
+    private final IndexWriter writer;
+    
+    private int maxQueryResults = DEFAULT_MAX_QUERY_RESULTS;
+
+    private boolean suffixMatch = false;
 
     
-    public LuceneMessageSearchIndex(MessageMapperFactory factory, Directory directory) throws CorruptIndexException, LockObtainFailedException, IOException {
-        this(factory, directory, false, true);
+    public LuceneMessageSearchIndex(MessageMapperFactory factory, MailboxId.Factory mailboxIdFactory, Directory directory) throws CorruptIndexException, LockObtainFailedException, IOException {
+        this(factory, mailboxIdFactory, directory, false, true);
     }
     
     
-    public LuceneMessageSearchIndex(MessageMapperFactory factory, Directory directory, boolean dropIndexOnStart, boolean lenient) throws CorruptIndexException, LockObtainFailedException, IOException {
+    public LuceneMessageSearchIndex(MessageMapperFactory factory, MailboxId.Factory mailboxIdFactory, Directory directory, boolean dropIndexOnStart, boolean lenient) throws CorruptIndexException, LockObtainFailedException, IOException {
         super(factory);
+        this.mailboxIdFactory = mailboxIdFactory;
         this.writer = new IndexWriter(directory,  createConfig(createAnalyzer(lenient), dropIndexOnStart));
     }
     
     
-    public LuceneMessageSearchIndex(MessageMapperFactory factory, IndexWriter writer) {
+    public LuceneMessageSearchIndex(MessageMapperFactory factory, MailboxId.Factory mailboxIdFactory, IndexWriter writer) {
         super(factory);
+        this.mailboxIdFactory = mailboxIdFactory;
         this.writer = writer;
     }
 
@@ -409,26 +420,47 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
      * @see org.apache.james.mailbox.store.search.MessageSearchIndex#search(org.apache.james.mailbox.MailboxSession, org.apache.james.mailbox.store.mail.model.Mailbox, org.apache.james.mailbox.model.SearchQuery)
      */
     public Iterator<Long> search(MailboxSession session, Mailbox mailbox, SearchQuery searchQuery) throws MailboxException {
-        Set<Long> uids = new LinkedHashSet<Long>();
+        MailboxId mailboxId = mailbox.getMailboxId();
+        Multimap<MailboxId, Long> results = 
+                searchMultimap(
+                    session, 
+                    MultimailboxesSearchQuery
+                        .from(searchQuery)
+                        .inMailboxes(mailboxId)
+                        .build());
+        return results.get(mailboxId).iterator();
+    }
+
+    @Override
+    public Map<MailboxId, Collection<Long>> search(MailboxSession session, MultimailboxesSearchQuery searchQuery) throws MailboxException {
+        return searchMultimap(session, searchQuery).asMap();
+    }
+    
+    private Multimap<MailboxId, Long> searchMultimap(MailboxSession session, MultimailboxesSearchQuery searchQuery) throws MailboxException {
+        Multimap<MailboxId, Long> results = LinkedHashMultimap.create();
         IndexSearcher searcher = null;
 
+        Query inMailboxes = buildQueryFromMailboxes(searchQuery.getMailboxIds());
+        
         try {
             searcher = new IndexSearcher(IndexReader.open(writer, true));
             BooleanQuery query = new BooleanQuery();
-            query.add(new TermQuery(new Term(MAILBOX_ID_FIELD, mailbox.getMailboxId().serialize())), BooleanClause.Occur.MUST);
+            query.add(inMailboxes, BooleanClause.Occur.MUST);
             // Not return flags documents
             query.add(new PrefixQuery(new Term(FLAGS_FIELD, "")), BooleanClause.Occur.MUST_NOT);
-            List<Criterion> crits = searchQuery.getCriterias();
+            List<Criterion> crits = searchQuery.getSearchQuery().getCriterias();
             for (Criterion crit : crits) {
-                query.add(createQuery(crit, mailbox, searchQuery.getRecentMessageUids()), BooleanClause.Occur.MUST);
+                query.add(createQuery(crit, inMailboxes, searchQuery.getSearchQuery().getRecentMessageUids()), BooleanClause.Occur.MUST);
             }
 
             // query for all the documents sorted as specified in the SearchQuery
-            TopDocs docs = searcher.search(query, null, maxQueryResults, createSort(searchQuery.getSorts()));
+            TopDocs docs = searcher.search(query, null, maxQueryResults, createSort(searchQuery.getSearchQuery().getSorts()));
             ScoreDoc[] sDocs = docs.scoreDocs;
             for (ScoreDoc sDoc : sDocs) {
-                long uid = Long.valueOf(searcher.doc(sDoc.doc).get(UID_FIELD));
-                uids.add(uid);
+                Document doc = searcher.doc(sDoc.doc);
+                long uid = Long.valueOf(doc.get(UID_FIELD));
+                MailboxId mailboxId = mailboxIdFactory.fromString(doc.get(MAILBOX_ID_FIELD));
+                results.put(mailboxId, uid);
             }
         } catch (IOException e) {
             throw new MailboxException("Unable to search the mailbox", e);
@@ -441,10 +473,22 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
                 }
             }
         }
-        return uids.iterator();
+        return results;
     }
-
    
+    private Query buildQueryFromMailboxes(ImmutableSet<MailboxId> mailboxIds) {
+        if (mailboxIds.isEmpty()) {
+            return new MatchAllDocsQuery();
+        }
+        BooleanQuery query = new BooleanQuery();
+        for (MailboxId id: mailboxIds) {
+            String idAsString = id.serialize();
+            query.add(new TermQuery(new Term(MAILBOX_ID_FIELD, idAsString)), BooleanClause.Occur.SHOULD);
+        }
+        return query;
+    }
+
+
     /**
      * Create a new {@link Document} for the given {@link MailboxMessage}. This Document does not contain any flags data. The {@link Flags} are stored in a seperate Document.
      * 
@@ -885,7 +929,7 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
      * @return query
      * @throws UnsupportedSearchException
      */
-    private Query createFlagQuery(String flag, boolean isSet, Mailbox mailbox, Collection<Long> recentUids) throws MailboxException, UnsupportedSearchException {
+    private Query createFlagQuery(String flag, boolean isSet, Query inMailboxes, Collection<Long> recentUids) throws MailboxException, UnsupportedSearchException {
         BooleanQuery query = new BooleanQuery();
         
         if (isSet) {   
@@ -898,7 +942,7 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
             
             query.add(bQuery, BooleanClause.Occur.MUST);
         }
-        query.add(new TermQuery(new Term(MAILBOX_ID_FIELD, mailbox.getMailboxId().serialize())), BooleanClause.Occur.MUST);
+        query.add(inMailboxes, BooleanClause.Occur.MUST);
         
         
         IndexSearcher searcher = null;
@@ -932,7 +976,7 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
             }
             return createUidQuery((UidCriterion) SearchQuery.uid(nRanges));
         } catch (IOException e) {
-            throw new MailboxException("Unable to search mailbox " + mailbox, e);
+            throw new MailboxException("Unable to search mailbox " + inMailboxes, e);
         } finally {
             if (searcher != null) {
                 try {
@@ -1114,26 +1158,26 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
      * @return query
      * @throws UnsupportedSearchException
      */
-    private Query createConjunctionQuery(SearchQuery.ConjunctionCriterion crit, Mailbox mailbox, Collection<Long> recentUids) throws UnsupportedSearchException, MailboxException {
+    private Query createConjunctionQuery(SearchQuery.ConjunctionCriterion crit, Query inMailboxes, Collection<Long> recentUids) throws UnsupportedSearchException, MailboxException {
         List<Criterion> crits = crit.getCriteria();
         BooleanQuery conQuery = new BooleanQuery();
         switch (crit.getType()) {
         case AND:
             for (Criterion criterion : crits) {
-                conQuery.add(createQuery(criterion, mailbox, recentUids), BooleanClause.Occur.MUST);
+                conQuery.add(createQuery(criterion, inMailboxes, recentUids), BooleanClause.Occur.MUST);
             }
             return conQuery;
         case OR:
             for (Criterion criterion : crits) {
-                conQuery.add(createQuery(criterion, mailbox, recentUids), BooleanClause.Occur.SHOULD);
+                conQuery.add(createQuery(criterion, inMailboxes, recentUids), BooleanClause.Occur.SHOULD);
             }
             return conQuery;
         case NOR:
             BooleanQuery nor = new BooleanQuery();
             for (Criterion criterion : crits) {
-                conQuery.add(createQuery(criterion, mailbox, recentUids), BooleanClause.Occur.SHOULD);
+                conQuery.add(createQuery(criterion, inMailboxes, recentUids), BooleanClause.Occur.SHOULD);
             }
-            nor.add(new TermQuery(new Term(MAILBOX_ID_FIELD, mailbox.getMailboxId().serialize())), BooleanClause.Occur.MUST);
+            nor.add(inMailboxes, BooleanClause.Occur.MUST);
 
             nor.add(conQuery, BooleanClause.Occur.MUST_NOT);
             return nor;
@@ -1150,7 +1194,7 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
      * @return query
      * @throws UnsupportedSearchException
      */
-    private Query createQuery(Criterion criterion, Mailbox mailbox, Collection<Long> recentUids) throws UnsupportedSearchException, MailboxException {
+    private Query createQuery(Criterion criterion, Query inMailboxes, Collection<Long> recentUids) throws UnsupportedSearchException, MailboxException {
         if (criterion instanceof SearchQuery.InternalDateCriterion) {
             SearchQuery.InternalDateCriterion crit = (SearchQuery.InternalDateCriterion) criterion;
             return createInternalDateQuery(crit);
@@ -1165,10 +1209,10 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
             return createUidQuery(crit);
         } else if (criterion instanceof SearchQuery.FlagCriterion) {
             FlagCriterion crit = (FlagCriterion) criterion;
-            return createFlagQuery(toString(crit.getFlag()), crit.getOperator().isSet(), mailbox, recentUids);
+            return createFlagQuery(toString(crit.getFlag()), crit.getOperator().isSet(), inMailboxes, recentUids);
         } else if (criterion instanceof SearchQuery.CustomFlagCriterion) {
             CustomFlagCriterion crit = (CustomFlagCriterion) criterion;
-            return createFlagQuery(crit.getFlag(), crit.getOperator().isSet(), mailbox, recentUids);
+            return createFlagQuery(crit.getFlag(), crit.getOperator().isSet(), inMailboxes, recentUids);
         } else if (criterion instanceof SearchQuery.TextCriterion) {
             SearchQuery.TextCriterion crit = (SearchQuery.TextCriterion) criterion;
             return createTextQuery(crit);
@@ -1176,7 +1220,7 @@ public class LuceneMessageSearchIndex extends ListeningMessageSearchIndex {
             return createAllQuery((AllCriterion) criterion);
         } else if (criterion instanceof SearchQuery.ConjunctionCriterion) {
             SearchQuery.ConjunctionCriterion crit = (SearchQuery.ConjunctionCriterion) criterion;
-            return createConjunctionQuery(crit, mailbox, recentUids);
+            return createConjunctionQuery(crit, inMailboxes, recentUids);
         } else if (criterion instanceof SearchQuery.ModSeqCriterion) {
             return createModSeqQuery((SearchQuery.ModSeqCriterion) criterion);
         }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMailboxMessageSearchIndexTest.java
----------------------------------------------------------------------
diff --git a/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMailboxMessageSearchIndexTest.java b/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMailboxMessageSearchIndexTest.java
index f9d5158..9b1417a 100644
--- a/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMailboxMessageSearchIndexTest.java
+++ b/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMailboxMessageSearchIndexTest.java
@@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThat;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Calendar;
+import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -33,6 +34,8 @@ import javax.mail.Flags;
 import javax.mail.Flags.Flag;
 
 import org.apache.james.mailbox.model.MailboxACL;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.SearchQuery.AddressType;
 import org.apache.james.mailbox.model.SearchQuery.DateResolution;
@@ -73,7 +76,7 @@ public class LuceneMailboxMessageSearchIndexTest {
     
     @Before
     public void setUp() throws Exception {
-        index = new LuceneMessageSearchIndex(null, new RAMDirectory(), true, useLenient());
+        index = new LuceneMessageSearchIndex(null, new TestId.Factory(), new RAMDirectory(), true, useLenient());
         index.setEnableSuffixMatch(true);
         Map<String, String> headersSubject = new HashMap<String, String>();
         headersSubject.put("Subject", "test (fwd)");
@@ -242,6 +245,35 @@ public class LuceneMailboxMessageSearchIndexTest {
         Iterator<Long> result = index.search(null, mailbox2, query);
         assertThat(result).containsExactly(1L);
     }
+
+    @Test
+    public void searchBodyInAllMailboxesShouldMatch() throws Exception {
+        SearchQuery query = new SearchQuery();
+        query.andCriteria(SearchQuery.bodyContains("My Body"));
+        Map<MailboxId, Collection<Long>> result = index.search(null, MultimailboxesSearchQuery.from(query).build());
+        assertThat(result).hasSize(2);
+        assertThat(result.get(mailbox.id)).containsExactly(1L);
+        assertThat(result.get(mailbox2.id)).containsExactly(1L);
+    }
+
+    @Test
+    public void searchBodyInSpecificMailboxesShouldMatch() throws Exception {
+        SearchQuery query = new SearchQuery();
+        query.andCriteria(SearchQuery.bodyContains("My Body"));
+        Map<MailboxId, Collection<Long>> result = index.search(null, 
+                MultimailboxesSearchQuery.from(query).inMailboxes(mailbox.id, mailbox3.id).build());
+        assertThat(result).hasSize(1);
+        assertThat(result.get(mailbox.id)).containsExactly(1L);
+    }
+
+
+    @Test
+    public void searchAllShouldMatchAllUserEmails() throws Exception {
+        SearchQuery query = new SearchQuery();
+        query.andCriteria(SearchQuery.all());
+        Map<MailboxId, Collection<Long>> result = index.search(null, MultimailboxesSearchQuery.from(query).build());
+        assertThat(result).hasSize(3);
+    }
     
     @Test
     public void flagSearchShouldMatch() throws Exception {

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndexTest.java
----------------------------------------------------------------------
diff --git a/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndexTest.java b/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndexTest.java
index db25e07..51f5ae7 100644
--- a/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndexTest.java
+++ b/mailbox/lucene/src/test/java/org/apache/james/mailbox/lucene/search/LuceneMessageSearchIndexTest.java
@@ -22,6 +22,7 @@ package org.apache.james.mailbox.lucene.search;
 import org.apache.james.mailbox.acl.SimpleGroupMembershipResolver;
 import org.apache.james.mailbox.acl.UnionMailboxACLResolver;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.inmemory.InMemoryId;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxManager;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxSessionMapperFactory;
 import org.apache.james.mailbox.store.JVMMailboxPathLocker;
@@ -41,7 +42,7 @@ public class LuceneMessageSearchIndexTest extends AbstractMessageSearchIndexTest
     @Override
     protected void initializeMailboxManager() throws Exception {
         MailboxSessionMapperFactory mapperFactory = new InMemoryMailboxSessionMapperFactory();
-        messageSearchIndex = new LuceneMessageSearchIndex(mapperFactory, new RAMDirectory());
+        messageSearchIndex = new LuceneMessageSearchIndex(mapperFactory, new InMemoryId.Factory(), new RAMDirectory());
         storeMailboxManager = new InMemoryMailboxManager(
             mapperFactory,
             new MockAuthenticator(),
@@ -56,7 +57,7 @@ public class LuceneMessageSearchIndexTest extends AbstractMessageSearchIndexTest
     /**
      * 15 tests out of 54 are failing
      */
-
+    
     @Ignore
     @Override
     public void uidShouldreturnEveryThing() throws Exception {
@@ -131,4 +132,14 @@ public class LuceneMessageSearchIndexTest extends AbstractMessageSearchIndexTest
     @Override
     public void modSeqEqualsShouldReturnUidsOfMessageHavingAGivenModSeq() throws Exception {
     }
+
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInTwoMailboxes() throws MailboxException {
+    }
+
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInAllMailboxes() throws MailboxException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryId.java
----------------------------------------------------------------------
diff --git a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryId.java b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryId.java
index 51602d5..c5139fc 100644
--- a/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryId.java
+++ b/mailbox/memory/src/main/java/org/apache/james/mailbox/inmemory/InMemoryId.java
@@ -22,6 +22,14 @@ import org.apache.james.mailbox.model.MailboxId;
 
 public class InMemoryId implements MailboxId {
 
+    public static class Factory implements MailboxId.Factory {
+        
+        @Override
+        public MailboxId fromString(String serialized) {
+            return of(Long.valueOf(serialized));
+        }
+    }
+    
     public static InMemoryId of(long value) {
         return new InMemoryId(value);
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/scanning-search/src/test/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndexTest.java
----------------------------------------------------------------------
diff --git a/mailbox/scanning-search/src/test/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndexTest.java b/mailbox/scanning-search/src/test/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndexTest.java
index c484c64..f034769 100644
--- a/mailbox/scanning-search/src/test/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndexTest.java
+++ b/mailbox/scanning-search/src/test/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndexTest.java
@@ -214,4 +214,29 @@ public class SimpleMessageSearchIndexTest extends AbstractMessageSearchIndexTest
     @Override
     public void sortShouldOrderMessages() throws Exception {
     }
+    
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInTwoMailboxes() throws MailboxException {
+    }
+
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInAllMailboxes() throws MailboxException {
+    }
+    
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageWithExpectedFromInTwoMailboxes() throws MailboxException {
+    }
+    
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageWithExpectedFromInAllMailboxes() throws MailboxException {
+    }
+    
+    @Ignore
+    @Override
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInOneMailbox() throws MailboxException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/LazyMessageSearchIndex.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/LazyMessageSearchIndex.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/LazyMessageSearchIndex.java
index 3d5d1ae..82704f9 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/LazyMessageSearchIndex.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/LazyMessageSearchIndex.java
@@ -18,14 +18,18 @@
  ****************************************************************/
 package org.apache.james.mailbox.store.search;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.exception.UnsupportedSearchException;
 import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.model.MessageRange;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.UpdatedFlags;
 import org.apache.james.mailbox.store.mail.MessageMapper.FetchType;
@@ -110,4 +114,10 @@ public class LazyMessageSearchIndex extends ListeningMessageSearchIndex {
     public void update(MailboxSession session, Mailbox mailbox, List<UpdatedFlags> updatedFlagsList) throws MailboxException {
         index.update(session, mailbox, updatedFlagsList);
     }
+    
+
+    @Override
+    public Map<MailboxId, Collection<Long>> search(MailboxSession session, MultimailboxesSearchQuery searchQuery) throws MailboxException {
+        throw new UnsupportedSearchException();
+    }
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/MessageSearchIndex.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/MessageSearchIndex.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/MessageSearchIndex.java
index c3ed9dd..b120b3a 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/MessageSearchIndex.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/MessageSearchIndex.java
@@ -19,10 +19,14 @@
 
 package org.apache.james.mailbox.store.search;
 
+import java.util.Collection;
 import java.util.Iterator;
+import java.util.Map;
 
 import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.model.MailboxId;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.store.mail.model.Mailbox;
 
@@ -37,11 +41,12 @@ public interface MessageSearchIndex {
     
     /**
      * Return all uids of the previous indexed {@link Mailbox}'s which match the {@link SearchQuery}
-     * 
-     * @param mailbox
-     * @param searchQuery
-     * @return Iterator on found uids
-     * @throws MailboxException
      */
     Iterator<Long> search(MailboxSession session, Mailbox mailbox, SearchQuery searchQuery) throws MailboxException;
+
+    /**
+     * Return all uids of all {@link Mailbox}'s the current user has access to which match the {@link SearchQuery}
+     */
+    Map<MailboxId, Collection<Long>> search(MailboxSession session, MultimailboxesSearchQuery searchQuery) throws MailboxException;
+
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndex.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndex.java b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndex.java
index 99d38f6..e4712c2 100644
--- a/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndex.java
+++ b/mailbox/store/src/main/java/org/apache/james/mailbox/store/search/SimpleMessageSearchIndex.java
@@ -18,8 +18,10 @@
  ****************************************************************/
 package org.apache.james.mailbox.store.search;
 
+import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
@@ -27,7 +29,10 @@ import javax.inject.Inject;
 
 import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.exception.UnsupportedSearchException;
+import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.model.MessageRange;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
 import org.apache.james.mailbox.model.SearchQuery.ConjunctionCriterion;
 import org.apache.james.mailbox.model.SearchQuery.Criterion;
@@ -109,4 +114,9 @@ public class SimpleMessageSearchIndex implements MessageSearchIndex {
 		}
     }
 
+    @Override
+    public Map<MailboxId, Collection<Long>> search(MailboxSession session, MultimailboxesSearchQuery searchQuery) throws MailboxException {
+        throw new UnsupportedSearchException();
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/store/src/test/java/org/apache/james/mailbox/store/TestId.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/test/java/org/apache/james/mailbox/store/TestId.java b/mailbox/store/src/test/java/org/apache/james/mailbox/store/TestId.java
index aa097c7..31b695c 100644
--- a/mailbox/store/src/test/java/org/apache/james/mailbox/store/TestId.java
+++ b/mailbox/store/src/test/java/org/apache/james/mailbox/store/TestId.java
@@ -22,6 +22,14 @@ import org.apache.james.mailbox.model.MailboxId;
 
 public class TestId implements MailboxId {
 
+    public static class Factory implements MailboxId.Factory {
+        
+        @Override
+        public MailboxId fromString(String serialized) {
+            return TestId.of(Long.valueOf(serialized));
+        }
+    }
+    
     public static TestId of(long id) {
         return new TestId(id);
     }

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mailbox/store/src/test/java/org/apache/james/mailbox/store/search/AbstractMessageSearchIndexTest.java
----------------------------------------------------------------------
diff --git a/mailbox/store/src/test/java/org/apache/james/mailbox/store/search/AbstractMessageSearchIndexTest.java b/mailbox/store/src/test/java/org/apache/james/mailbox/store/search/AbstractMessageSearchIndexTest.java
index d52cbfe..d2ab3d8 100644
--- a/mailbox/store/src/test/java/org/apache/james/mailbox/store/search/AbstractMessageSearchIndexTest.java
+++ b/mailbox/store/src/test/java/org/apache/james/mailbox/store/search/AbstractMessageSearchIndexTest.java
@@ -19,16 +19,19 @@
 
 package org.apache.james.mailbox.store.search;
 
-import static org.assertj.core.api.Assertions.assertThat;
-
+import java.util.Collection;
 import java.util.Date;
+import java.util.Map;
 
 import javax.mail.Flags;
 
 import org.apache.james.mailbox.MailboxSession;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.model.MailboxPath;
+import org.apache.james.mailbox.model.MultimailboxesSearchQuery;
 import org.apache.james.mailbox.model.SearchQuery;
+import org.apache.james.mailbox.model.SearchQuery.AddressType;
 import org.apache.james.mailbox.store.StoreMailboxManager;
 import org.apache.james.mailbox.store.StoreMessageManager;
 import org.apache.james.mailbox.store.mail.model.Mailbox;
@@ -39,6 +42,8 @@ import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Lists;
 
+import static org.assertj.core.api.Assertions.assertThat;
+
 public abstract class AbstractMessageSearchIndexTest {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(AbstractMessageSearchIndexTest.class);
@@ -46,6 +51,7 @@ public abstract class AbstractMessageSearchIndexTest {
     protected MessageSearchIndex messageSearchIndex;
     protected StoreMailboxManager storeMailboxManager;
     private Mailbox mailbox;
+    private Mailbox mailbox2;
     private MailboxSession session;
 
     @Before
@@ -54,13 +60,18 @@ public abstract class AbstractMessageSearchIndexTest {
 
         session = storeMailboxManager.createSystemSession("benwa", LOGGER);
 
-        storeMailboxManager.createMailbox(new MailboxPath("#private", "benwa", "INBOX"), session);
-        StoreMessageManager messageManager = (StoreMessageManager) storeMailboxManager.getMailbox(new MailboxPath("#private", "benwa", "INBOX"), session);
-        mailbox = messageManager.getMailboxEntity();
+        MailboxPath inboxPath = new MailboxPath("#private", "benwa", "INBOX");
+        storeMailboxManager.createMailbox(inboxPath, session);
+        StoreMessageManager inboxMessageManager = (StoreMessageManager) storeMailboxManager.getMailbox(inboxPath, session);
+        MailboxPath myFolderPath = new MailboxPath("#private", "benwa", "MyFolder");
+        storeMailboxManager.createMailbox(myFolderPath, session);
+        StoreMessageManager myFolderMessageManager = (StoreMessageManager) storeMailboxManager.getMailbox(myFolderPath, session);
+        mailbox = inboxMessageManager.getMailboxEntity();
+        mailbox2 = myFolderMessageManager.getMailboxEntity();
 
         // sentDate: Wed, 3 Jun 2015 09:05:46 +0000
         // Internal date : 2014/01/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/spamMail.eml"),
             new Date(1388617200000L),
             session,
@@ -68,7 +79,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.DELETED));
         // sentDate: Thu, 4 Jun 2015 09:23:37 +0000
         // Internal date : 2014/02/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/mail1.eml"),
             new Date(1391295600000L),
             session,
@@ -76,7 +87,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.ANSWERED));
         // sentDate: Thu, 4 Jun 2015 09:27:37 +0000
         // Internal date : 2014/03/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/mail2.eml"),
             new Date(1393714800000L),
             session,
@@ -84,7 +95,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.DRAFT));
         // sentDate: Tue, 2 Jun 2015 08:16:19 +0000
         // Internal date : 2014/05/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/mail3.eml"),
             new Date(1398981600000L),
             session,
@@ -92,7 +103,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.RECENT));
         // sentDate: Fri, 15 May 2015 06:35:59 +0000
         // Internal date : 2014/04/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/mail4.eml"),
             new Date(1396389600000L),
             session,
@@ -100,7 +111,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.FLAGGED));
         // sentDate: Wed, 03 Jun 2015 19:14:32 +0000
         // Internal date : 2014/06/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/pgpSignedMail.eml"),
             new Date(1401660000000L),
             session,
@@ -108,7 +119,7 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags(Flags.Flag.SEEN));
         // sentDate: Thu, 04 Jun 2015 07:36:08 +0000
         // Internal date : 2014/07/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/htmlMail.eml"),
             new Date(1404252000000L),
             session,
@@ -116,21 +127,28 @@ public abstract class AbstractMessageSearchIndexTest {
             new Flags());
         // sentDate: Thu, 4 Jun 2015 06:08:41 +0200
         // Internal date : 2014/08/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/mail.eml"),
             new Date(1406930400000L),
             session,
             true,
             new Flags("Hello"));
+        // sentDate: Thu, 4 Jun 2015 06:08:41 +0200
+        // Internal date : 2014/08/02 00:00:00.000
+        myFolderMessageManager.appendMessage(
+            ClassLoader.getSystemResourceAsStream("eml/mail.eml"),
+            new Date(1406930400000L),
+            session,
+            true,
+            new Flags(Flags.Flag.SEEN));
         // sentDate: Tue, 2 Jun 2015 12:00:55 +0200
         // Internal date : 2014/09/02 00:00:00.000
-        messageManager.appendMessage(
+        inboxMessageManager.appendMessage(
             ClassLoader.getSystemResourceAsStream("eml/frnog.eml"),
             new Date(1409608800000L),
             session,
             true,
             new Flags("Hello you"));
-
         await();
     }
 
@@ -224,8 +242,76 @@ public abstract class AbstractMessageSearchIndexTest {
         assertThat(messageSearchIndex.search(session, mailbox, searchQuery))
             .containsOnly(6L);
     }
+    
+    @Test
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInAllMailboxes() throws MailboxException {
+        SearchQuery searchQuery = new SearchQuery();
+        searchQuery.andCriteria(SearchQuery.flagIsSet(Flags.Flag.SEEN));
+        Map<MailboxId, Collection<Long>> actual = messageSearchIndex.search(session, MultimailboxesSearchQuery.from(searchQuery).build());
+        assertThat(actual.entrySet()).hasSize(2);
+        assertThat(actual.get(mailbox.getMailboxId())).containsExactly(6L);
+        assertThat(actual.get(mailbox2.getMailboxId())).containsExactly(1L);
+    }
 
     @Test
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInOneMailbox() throws MailboxException {
+        SearchQuery searchQuery = new SearchQuery();
+        searchQuery.andCriteria(SearchQuery.flagIsSet(Flags.Flag.SEEN));
+        MultimailboxesSearchQuery query = 
+                MultimailboxesSearchQuery
+                    .from(searchQuery)
+                    .inMailboxes(mailbox.getMailboxId())
+                    .build();
+        Map<MailboxId, Collection<Long>> actual = messageSearchIndex.search(session, query);
+        assertThat(actual.entrySet()).hasSize(1);
+        assertThat(actual.get(mailbox.getMailboxId())).containsExactly(6L);
+    }
+
+    @Test
+    public void multimailboxSearchShouldReturnUidOfMessageWithExpectedFromInTwoMailboxes() throws MailboxException {
+        SearchQuery searchQuery = new SearchQuery();
+        searchQuery.andCriteria(SearchQuery.address(AddressType.From, "murari"));
+        MultimailboxesSearchQuery query = 
+                MultimailboxesSearchQuery
+                    .from(searchQuery)
+                    .inMailboxes(mailbox.getMailboxId(), mailbox2.getMailboxId())
+                    .build();
+        Map<MailboxId, Collection<Long>> actual = messageSearchIndex.search(session, query);
+        assertThat(actual.entrySet()).hasSize(2);
+        assertThat(actual.get(mailbox.getMailboxId())).containsExactly(8L);
+        assertThat(actual.get(mailbox2.getMailboxId())).containsExactly(1L);
+    }
+
+    @Test
+    public void multimailboxSearchShouldReturnUidOfMessageWithExpectedFromInAllMailboxes() throws MailboxException {
+        SearchQuery searchQuery = new SearchQuery();
+        searchQuery.andCriteria(SearchQuery.address(AddressType.From, "murari"));
+        MultimailboxesSearchQuery query = 
+                MultimailboxesSearchQuery
+                    .from(searchQuery)
+                    .build();
+        Map<MailboxId, Collection<Long>> actual = messageSearchIndex.search(session, query);
+        assertThat(actual.entrySet()).hasSize(2);
+        assertThat(actual.get(mailbox.getMailboxId())).containsExactly(8L);
+        assertThat(actual.get(mailbox2.getMailboxId())).containsExactly(1L);
+    }
+
+    @Test
+    public void multimailboxSearchShouldReturnUidOfMessageMarkedAsSeenInTwoMailboxes() throws MailboxException {
+        SearchQuery searchQuery = new SearchQuery();
+        searchQuery.andCriteria(SearchQuery.flagIsSet(Flags.Flag.SEEN));
+        MultimailboxesSearchQuery query = 
+                MultimailboxesSearchQuery
+                    .from(searchQuery)
+                    .inMailboxes(mailbox.getMailboxId(), mailbox2.getMailboxId())
+                    .build();
+        Map<MailboxId, Collection<Long>> actual = messageSearchIndex.search(session, query);
+        assertThat(actual.entrySet()).hasSize(2);
+        assertThat(actual.get(mailbox.getMailboxId())).containsExactly(6L);
+        assertThat(actual.get(mailbox2.getMailboxId())).containsExactly(1L);
+    }
+    
+    @Test
     public void flagIsSetShouldReturnUidsOfMessageContainingAGivenUserFlag() throws MailboxException {
         SearchQuery searchQuery = new SearchQuery();
         searchQuery.andCriteria(SearchQuery.flagIsSet("Hello"));

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mpt/impl/imap-mailbox/core/src/main/resources/org/apache/james/imap/scripts/UidSearchAtomsIndexer.test
----------------------------------------------------------------------
diff --git a/mpt/impl/imap-mailbox/core/src/main/resources/org/apache/james/imap/scripts/UidSearchAtomsIndexer.test b/mpt/impl/imap-mailbox/core/src/main/resources/org/apache/james/imap/scripts/UidSearchAtomsIndexer.test
index 39bd1e3..320cbe9 100644
--- a/mpt/impl/imap-mailbox/core/src/main/resources/org/apache/james/imap/scripts/UidSearchAtomsIndexer.test
+++ b/mpt/impl/imap-mailbox/core/src/main/resources/org/apache/james/imap/scripts/UidSearchAtomsIndexer.test
@@ -2211,4 +2211,39 @@ S: A135 OK SEARCH completed\.
 
 # C: A139 UID SEARCH NEW
 # S: \* SEARCH 23 24 28 29 33 34 38 39
-# S: A139 OK SEARCH completed\.
\ No newline at end of file
+# S: A139 OK SEARCH completed\.
+
+C: A200 CREATE secondmailbox
+S: A200 OK CREATE completed\.
+
+C: A201 SELECT secondmailbox
+S: \* FLAGS \(\\Answered \\Deleted \\Draft \\Flagged \\Seen\)
+S: \* 0 EXISTS
+S: \* 0 RECENT
+S: \* OK \[UIDVALIDITY (.)*
+S: \* OK \[PERMANENTFLAGS \(\\Answered \\Deleted \\Draft \\Flagged \\\Seen( \\\*)?\)\](.)*
+S: \* OK \[HIGHESTMODSEQ \d+\].*
+S: \* OK \[UIDNEXT 1\].*
+S: A201 OK \[READ-WRITE\] SELECT completed\.
+
+C: A202 APPEND secondmailbox {185+}
+C: From: Timothy Tayler <ti...@example.org>
+C: To: Samual Smith <sa...@example.org>
+C: Date: Thu, 14 Feb 2008 12:00:00 +0000 (GMT)
+C: Subject: A Simple Email
+C: 
+C: This is a very simple email.
+C: 
+S: \* 1 EXISTS
+S: \* 1 RECENT
+S: A202 OK (\[.+\] )?APPEND completed\.
+
+LOG INFO Waiting for ElasticSearch to index data
+
+WAIT 1000
+
+LOG INFO Performing tests about search
+
+C: A203 UID SEARCH FROM "Timothy"
+S: \* SEARCH 1
+S: A203 OK SEARCH completed\.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/mpt/impl/imap-mailbox/elasticsearch/src/test/java/org/apache/james/mpt/imapmailbox/elasticsearch/host/ElasticSearchHostSystem.java
----------------------------------------------------------------------
diff --git a/mpt/impl/imap-mailbox/elasticsearch/src/test/java/org/apache/james/mpt/imapmailbox/elasticsearch/host/ElasticSearchHostSystem.java b/mpt/impl/imap-mailbox/elasticsearch/src/test/java/org/apache/james/mpt/imapmailbox/elasticsearch/host/ElasticSearchHostSystem.java
index 6aaf843..b354514 100644
--- a/mpt/impl/imap-mailbox/elasticsearch/src/test/java/org/apache/james/mpt/imapmailbox/elasticsearch/host/ElasticSearchHostSystem.java
+++ b/mpt/impl/imap-mailbox/elasticsearch/src/test/java/org/apache/james/mpt/imapmailbox/elasticsearch/host/ElasticSearchHostSystem.java
@@ -44,6 +44,7 @@ import org.apache.james.mailbox.elasticsearch.query.QueryConverter;
 import org.apache.james.mailbox.elasticsearch.search.ElasticSearchSearcher;
 import org.apache.james.mailbox.elasticsearch.utils.TestingClientProvider;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.inmemory.InMemoryId;
 import org.apache.james.mailbox.inmemory.InMemoryMailboxSessionMapperFactory;
 import org.apache.james.mailbox.model.MailboxPath;
 import org.apache.james.mailbox.store.MockAuthenticator;
@@ -105,7 +106,7 @@ public class ElasticSearchHostSystem extends JamesImapHostSystem {
         ElasticSearchListeningMessageSearchIndex searchIndex = new ElasticSearchListeningMessageSearchIndex(
             factory,
             new ElasticSearchIndexer(client, new DeleteByQueryPerformer(client, Executors.newSingleThreadExecutor())),
-            new ElasticSearchSearcher(client, new QueryConverter(new CriterionConverter())),
+            new ElasticSearchSearcher(client, new QueryConverter(new CriterionConverter()), new InMemoryId.Factory()),
             new MessageToElasticSearchJson(new DefaultTextExtractor()));
 
         MailboxACLResolver aclResolver = new UnionMailboxACLResolver();

http://git-wip-us.apache.org/repos/asf/james-project/blob/085cf65e/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
----------------------------------------------------------------------
diff --git a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
index 853ce0b..7d358f3 100644
--- a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
+++ b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraMailboxModule.java
@@ -25,12 +25,14 @@ import org.apache.james.backends.cassandra.components.CassandraModule;
 import org.apache.james.mailbox.MailboxManager;
 import org.apache.james.mailbox.MailboxPathLocker;
 import org.apache.james.mailbox.SubscriptionManager;
+import org.apache.james.mailbox.cassandra.CassandraId;
 import org.apache.james.mailbox.cassandra.CassandraMailboxManager;
 import org.apache.james.mailbox.cassandra.CassandraMailboxSessionMapperFactory;
 import org.apache.james.mailbox.cassandra.CassandraSubscriptionManager;
 import org.apache.james.mailbox.cassandra.mail.CassandraModSeqProvider;
 import org.apache.james.mailbox.cassandra.mail.CassandraUidProvider;
 import org.apache.james.mailbox.exception.MailboxException;
+import org.apache.james.mailbox.model.MailboxId;
 import org.apache.james.mailbox.store.Authenticator;
 import org.apache.james.mailbox.store.MailboxSessionMapperFactory;
 import org.apache.james.mailbox.store.NoMailboxPathLocker;
@@ -60,6 +62,7 @@ public class CassandraMailboxModule extends AbstractModule {
         bind(CassandraModSeqProvider.class).in(Scopes.SINGLETON);
         bind(CassandraUidProvider.class).in(Scopes.SINGLETON);
         bind(UserRepositoryAuthenticator.class).in(Scopes.SINGLETON);
+        bind(CassandraId.Factory.class).in(Scopes.SINGLETON);
 
         bind(MessageMapperFactory.class).to(CassandraMailboxSessionMapperFactory.class);
         bind(MailboxMapperFactory.class).to(CassandraMailboxSessionMapperFactory.class);
@@ -72,6 +75,7 @@ public class CassandraMailboxModule extends AbstractModule {
         bind(MailboxPathLocker.class).to(NoMailboxPathLocker.class);
         bind(Authenticator.class).to(UserRepositoryAuthenticator.class);
         bind(MailboxManager.class).to(CassandraMailboxManager.class);
+        bind(MailboxId.Factory.class).to(CassandraId.Factory.class);
 
         Multibinder<CassandraModule> cassandraDataDefinitions = Multibinder.newSetBinder(binder(), CassandraModule.class);
         cassandraDataDefinitions.addBinding().to(org.apache.james.mailbox.cassandra.modules.CassandraAclModule.class);


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