You are viewing a plain text version of this content. The canonical link for it is here.
Posted to server-dev@james.apache.org by bt...@apache.org on 2018/05/28 01:06:20 UTC

[10/13] james-project git commit: JAMES-2403 ElasticSearch implementation of QuotaSearcher

JAMES-2403 ElasticSearch implementation of QuotaSearcher


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

Branch: refs/heads/master
Commit: dd95f7c5567cecc99ca38b62238881811fb49d48
Parents: f534186
Author: Raphael Ouazana <ra...@linagora.com>
Authored: Wed May 23 18:17:01 2018 +0200
Committer: benwa <bt...@linagora.com>
Committed: Mon May 28 08:05:35 2018 +0700

----------------------------------------------------------------------
 .../plugin/quota-search-elasticsearch/pom.xml   |  35 ++++++
 .../ElasticSearchQuotaSearcher.java             |  90 +++++++++++++++
 .../elasticsearch/QuotaQueryConverter.java      | 103 +++++++++++++++++
 ...ticSearchQuotaSearchTestSystemExtension.java | 112 +++++++++++++++++++
 .../ElasticSearchQuotaSearcherTest.java         |  28 +++++
 .../elasticsearch/QuotaQueryConverterTest.java  |  83 ++++++++++++++
 .../MemoryQuotaSearchTestSystemExtension.java   |   4 +-
 .../quota/search/QuotaSearchTestSystem.java     |   8 +-
 .../quota/search/QuotaSearcherContract.java     |  10 ++
 9 files changed, 471 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/pom.xml
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/pom.xml b/mailbox/plugin/quota-search-elasticsearch/pom.xml
index 79b9ef1..7509c9a 100644
--- a/mailbox/plugin/quota-search-elasticsearch/pom.xml
+++ b/mailbox/plugin/quota-search-elasticsearch/pom.xml
@@ -56,11 +56,41 @@
         </dependency>
         <dependency>
             <groupId>${project.groupId}</groupId>
+            <artifactId>apache-james-mailbox-memory</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>apache-james-mailbox-memory</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
             <artifactId>apache-james-mailbox-quota-search</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>apache-james-mailbox-quota-search</artifactId>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>james-core</artifactId>
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>james-server-data-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>${project.groupId}</groupId>
+            <artifactId>james-server-data-memory</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>com.fasterxml.jackson.core</groupId>
             <artifactId>jackson-databind</artifactId>
         </dependency>
@@ -89,6 +119,11 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.mockito</groupId>
+            <artifactId>mockito-core</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>net.javacrumbs.json-unit</groupId>
             <artifactId>json-unit-fluent</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcher.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcher.java b/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcher.java
new file mode 100644
index 0000000..7e0857b
--- /dev/null
+++ b/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcher.java
@@ -0,0 +1,90 @@
+/****************************************************************
+ * 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.quota.search.elasticsearch;
+
+import static org.apache.james.quota.search.elasticsearch.QuotaRatioElasticSearchConstants.QUOTA_RATIO_TYPE;
+import static org.apache.james.quota.search.elasticsearch.json.JsonMessageConstants.USER;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Stream;
+
+import org.apache.james.backends.es.AliasName;
+import org.apache.james.backends.es.search.ScrollIterable;
+import org.apache.james.core.User;
+import org.apache.james.quota.search.QuotaQuery;
+import org.apache.james.quota.search.QuotaSearcher;
+import org.elasticsearch.action.search.SearchRequestBuilder;
+import org.elasticsearch.client.Client;
+import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.search.sort.SortBuilders;
+import org.elasticsearch.search.sort.SortOrder;
+
+import com.github.steveash.guavate.Guavate;
+
+public class ElasticSearchQuotaSearcher implements QuotaSearcher {
+    private static final TimeValue TIMEOUT = new TimeValue(60000);
+
+    private final Client client;
+    private final AliasName readAlias;
+    private final QuotaQueryConverter quotaQueryConverter;
+
+    public ElasticSearchQuotaSearcher(Client client, AliasName readAlias) {
+        this.client = client;
+        this.readAlias = readAlias;
+        this.quotaQueryConverter = new QuotaQueryConverter();
+    }
+
+    @Override
+    public List<User> search(QuotaQuery query) {
+        Stream<User> results = new ScrollIterable(client, prepareSearch(query))
+            .stream()
+            .flatMap(searchResponse -> Arrays.stream(searchResponse.getHits()
+                .getHits()))
+            .map(hit -> hit.field(USER))
+            .map(field -> (String) field.getValue())
+            .map(User::fromUsername)
+            .skip(query.getOffset().getValue());
+
+        return query.getLimit().getValue()
+            .map(results::limit)
+            .orElse(results)
+            .collect(Guavate.toImmutableList());
+    }
+
+    public SearchRequestBuilder prepareSearch(QuotaQuery query) {
+        SearchRequestBuilder searchRequestBuilder = client.prepareSearch(readAlias.getValue())
+            .setTypes(QUOTA_RATIO_TYPE.getValue())
+            .setScroll(TIMEOUT)
+            .addFields(USER)
+            .setQuery(quotaQueryConverter.from(query));
+
+        query.getLimit()
+            .getValue()
+            .ifPresent(searchRequestBuilder::setSize);
+
+        searchRequestBuilder.addSort(
+            SortBuilders.fieldSort(USER)
+                .order(SortOrder.ASC));
+
+        return searchRequestBuilder;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverter.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverter.java b/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverter.java
new file mode 100644
index 0000000..b02d8ac
--- /dev/null
+++ b/mailbox/plugin/quota-search-elasticsearch/src/main/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverter.java
@@ -0,0 +1,103 @@
+/****************************************************************
+ * 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.quota.search.elasticsearch;
+
+import static org.apache.james.quota.search.elasticsearch.json.JsonMessageConstants.DOMAIN;
+import static org.apache.james.quota.search.elasticsearch.json.JsonMessageConstants.QUOTA_RATIO;
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
+import static org.elasticsearch.index.query.QueryBuilders.termQuery;
+
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.james.quota.search.QuotaClause;
+import org.apache.james.quota.search.QuotaClause.And;
+import org.apache.james.quota.search.QuotaClause.HasDomain;
+import org.apache.james.quota.search.QuotaClause.LessThan;
+import org.apache.james.quota.search.QuotaClause.MoreThan;
+import org.apache.james.quota.search.QuotaQuery;
+import org.elasticsearch.index.query.BoolQueryBuilder;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.elasticsearch.index.query.RangeQueryBuilder;
+import org.elasticsearch.index.query.TermQueryBuilder;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMap.Builder;
+
+public class QuotaQueryConverter {
+    private final Map<Class<? extends QuotaClause>, Function<QuotaClause, QueryBuilder>> clauseConverter;
+
+    public QuotaQueryConverter() {
+        Builder<Class<? extends QuotaClause>, Function<QuotaClause, QueryBuilder>> builder = ImmutableMap.builder();
+        
+        builder.put(HasDomain.class, this::convertHasDomain);
+        builder.put(And.class, this::disableNestedAnd);
+        builder.put(MoreThan.class, this::convertMoreThan);
+        builder.put(LessThan.class, this::convertLessThan);
+
+        clauseConverter = builder.build();
+    }
+
+    public QueryBuilder from(QuotaQuery query) {
+        List<QuotaClause> clauses = query.getClause().getClauses();
+        if (clauses.isEmpty()) {
+            return matchAllQuery();
+        }
+        if (clauses.size() == 1) {
+            return singleClauseAsESQuery(clauses.get(0));
+        }
+        
+        return clausesAsAndESQuery(clauses);
+    }
+
+    private BoolQueryBuilder clausesAsAndESQuery(List<QuotaClause> clauses) {
+        BoolQueryBuilder boolQueryBuilder = new BoolQueryBuilder();
+        clauses.stream()
+            .map(this::singleClauseAsESQuery)
+            .forEach(boolQueryBuilder::must);
+        return boolQueryBuilder;
+    }
+
+    private QueryBuilder disableNestedAnd(QuotaClause clause) {
+        throw new IllegalArgumentException("Nested \"And\" clauses are not supported");
+    }
+
+    private TermQueryBuilder convertHasDomain(QuotaClause clause) {
+        HasDomain hasDomain = (HasDomain) clause;
+        return termQuery(DOMAIN, hasDomain.getDomain().asString());
+    }
+
+    private RangeQueryBuilder convertMoreThan(QuotaClause clause) {
+        MoreThan moreThan = (MoreThan) clause;
+        return rangeQuery(QUOTA_RATIO).gte(moreThan.getQuotaBoundary().getRatio());
+    }
+
+    private RangeQueryBuilder convertLessThan(QuotaClause clause) {
+        LessThan lessThan = (LessThan) clause;
+        return rangeQuery(QUOTA_RATIO).lte(lessThan.getQuotaBoundary().getRatio());
+    }
+
+    private QueryBuilder singleClauseAsESQuery(QuotaClause clause) {
+        return clauseConverter.get(clause.getClass()).apply(clause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearchTestSystemExtension.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearchTestSystemExtension.java b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearchTestSystemExtension.java
new file mode 100644
index 0000000..d4dc2a2
--- /dev/null
+++ b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearchTestSystemExtension.java
@@ -0,0 +1,112 @@
+/****************************************************************
+ * 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.quota.search.elasticsearch;
+
+import static org.mockito.Mockito.mock;
+
+import java.util.concurrent.Executors;
+
+import org.apache.james.backends.es.ElasticSearchIndexer;
+import org.apache.james.backends.es.EmbeddedElasticSearch;
+import org.apache.james.backends.es.IndexCreationFactory;
+import org.apache.james.backends.es.NodeMappingFactory;
+import org.apache.james.backends.es.utils.TestingClientProvider;
+import org.apache.james.dnsservice.api.DNSService;
+import org.apache.james.domainlist.memory.MemoryDomainList;
+import org.apache.james.mailbox.acl.SimpleGroupMembershipResolver;
+import org.apache.james.mailbox.inmemory.manager.InMemoryIntegrationResources;
+import org.apache.james.mailbox.mock.MockMailboxSession;
+import org.apache.james.quota.search.QuotaSearchTestSystem;
+import org.apache.james.quota.search.elasticsearch.events.ElasticSearchQuotaMailboxListener;
+import org.apache.james.quota.search.elasticsearch.json.QuotaRatioToElasticSearchJson;
+import org.apache.james.user.memory.MemoryUsersRepository;
+import org.elasticsearch.client.Client;
+import org.junit.jupiter.api.extension.AfterEachCallback;
+import org.junit.jupiter.api.extension.BeforeEachCallback;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.api.extension.ParameterContext;
+import org.junit.jupiter.api.extension.ParameterResolutionException;
+import org.junit.jupiter.api.extension.ParameterResolver;
+import org.junit.rules.TemporaryFolder;
+
+public class ElasticSearchQuotaSearchTestSystemExtension implements ParameterResolver, BeforeEachCallback, AfterEachCallback {
+
+    private TemporaryFolder temporaryFolder = new TemporaryFolder();
+    private EmbeddedElasticSearch embeddedElasticSearch = new EmbeddedElasticSearch(temporaryFolder, QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_INDEX);
+
+    @Override
+    public boolean supportsParameter(ParameterContext parameterContext, ExtensionContext extensionContext) throws ParameterResolutionException {
+        return (parameterContext.getParameter().getType() == QuotaSearchTestSystem.class);
+    }
+
+    @Override
+    public Object resolveParameter(ParameterContext parameterContext, ExtensionContext extensionContext) throws ParameterResolutionException {
+        try {
+            Client client = NodeMappingFactory.applyMapping(
+                new IndexCreationFactory()
+                    .useIndex(QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_INDEX)
+                    .addAlias(QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_READ_ALIAS)
+                    .addAlias(QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_WRITE_ALIAS)
+                    .createIndexAndAliases(new TestingClientProvider(embeddedElasticSearch.getNode()).get()),
+                QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_INDEX,
+                QuotaRatioElasticSearchConstants.QUOTA_RATIO_TYPE,
+                QuotaRatioMappingFactory.getMappingContent());
+
+            InMemoryIntegrationResources.Resources resources = new InMemoryIntegrationResources().createResources(new SimpleGroupMembershipResolver());
+
+            MemoryUsersRepository usersRepository = MemoryUsersRepository.withVirtualHosting();
+
+            DNSService dnsService = mock(DNSService.class);
+            MemoryDomainList domainList = new MemoryDomainList(dnsService);
+            usersRepository.setDomainList(domainList);
+
+            ElasticSearchQuotaMailboxListener listener = new ElasticSearchQuotaMailboxListener(
+                new ElasticSearchIndexer(client, Executors.newSingleThreadExecutor(),
+                    QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_WRITE_ALIAS,
+                    QuotaRatioElasticSearchConstants.QUOTA_RATIO_TYPE),
+                new QuotaRatioToElasticSearchJson());
+
+            resources.getMailboxManager()
+                .addGlobalListener(listener, new MockMailboxSession("ANY"));
+
+            return new QuotaSearchTestSystem(
+                resources.getMaxQuotaManager(),
+                resources.getMailboxManager(),
+                new ElasticSearchQuotaSearcher(client,
+                    QuotaRatioElasticSearchConstants.DEFAULT_QUOTA_RATIO_READ_ALIAS),
+                usersRepository, domainList,
+                () -> embeddedElasticSearch.awaitForElasticSearch());
+        } catch (Exception e) {
+            throw new ParameterResolutionException("Error while resolving parameter", e);
+        }
+    }
+
+    @Override
+    public void beforeEach(ExtensionContext context) throws Exception {
+        temporaryFolder.create();
+        embeddedElasticSearch.before();
+    }
+
+    @Override
+    public void afterEach(ExtensionContext context) {
+        embeddedElasticSearch.after();
+        temporaryFolder.delete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcherTest.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcherTest.java b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcherTest.java
new file mode 100644
index 0000000..ad26532
--- /dev/null
+++ b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/ElasticSearchQuotaSearcherTest.java
@@ -0,0 +1,28 @@
+/****************************************************************
+ * 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.quota.search.elasticsearch;
+
+import org.apache.james.quota.search.QuotaSearcherContract;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+@ExtendWith(ElasticSearchQuotaSearchTestSystemExtension.class)
+public class ElasticSearchQuotaSearcherTest implements QuotaSearcherContract {
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverterTest.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverterTest.java b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverterTest.java
new file mode 100644
index 0000000..86daa88
--- /dev/null
+++ b/mailbox/plugin/quota-search-elasticsearch/src/test/java/org/apache/james/quota/search/elasticsearch/QuotaQueryConverterTest.java
@@ -0,0 +1,83 @@
+/****************************************************************
+ * 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.quota.search.elasticsearch;
+
+import static org.apache.james.quota.search.elasticsearch.json.JsonMessageConstants.QUOTA_RATIO;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
+import static org.elasticsearch.index.query.QueryBuilders.rangeQuery;
+import static org.elasticsearch.index.query.QueryBuilders.termQuery;
+
+import org.apache.james.core.Domain;
+import org.apache.james.quota.search.QuotaBoundary;
+import org.apache.james.quota.search.QuotaQuery;
+import org.apache.james.quota.search.elasticsearch.json.JsonMessageConstants;
+import org.elasticsearch.index.query.QueryBuilder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+class QuotaQueryConverterTest {
+    private QuotaQueryConverter testee;
+
+    @BeforeEach
+    void setup() {
+        testee = new QuotaQueryConverter();
+    }
+
+    @Test
+    void fromShouldReturnMatchAllWhenEmptyClauses() {
+        QuotaQuery query = QuotaQuery.builder().build();
+        QueryBuilder expected = matchAllQuery();
+
+        QueryBuilder actual = testee.from(query);
+
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    void fromShouldReturnDomainMatchWhenOnlyDomain() {
+        QuotaQuery query = QuotaQuery.builder().hasDomain(Domain.of("my.tld")).build();
+        QueryBuilder expected = termQuery(JsonMessageConstants.DOMAIN, "my.tld");
+
+        QueryBuilder actual = testee.from(query);
+
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    void fromShouldReturnQuotaRatioMatchWhenLessThan() {
+        QuotaQuery query = QuotaQuery.builder().lessThan(new QuotaBoundary(0.1)).build();
+        QueryBuilder expected = rangeQuery(QUOTA_RATIO).lte(0.1);
+
+        QueryBuilder actual = testee.from(query);
+
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+    @Test
+    void fromShouldReturnQuotaRatioMatchWhenMoreThan() {
+        QuotaQuery query = QuotaQuery.builder().moreThan(new QuotaBoundary(0.1)).build();
+        QueryBuilder expected = rangeQuery(QUOTA_RATIO).gte(0.1);
+
+        QueryBuilder actual = testee.from(query);
+
+        assertThat(actual).isEqualToComparingFieldByField(expected);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search-scanning/src/test/java/org/apache/james/quota/search/scanning/MemoryQuotaSearchTestSystemExtension.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search-scanning/src/test/java/org/apache/james/quota/search/scanning/MemoryQuotaSearchTestSystemExtension.java b/mailbox/plugin/quota-search-scanning/src/test/java/org/apache/james/quota/search/scanning/MemoryQuotaSearchTestSystemExtension.java
index 6bd1df3..27b08d2 100644
--- a/mailbox/plugin/quota-search-scanning/src/test/java/org/apache/james/quota/search/scanning/MemoryQuotaSearchTestSystemExtension.java
+++ b/mailbox/plugin/quota-search-scanning/src/test/java/org/apache/james/quota/search/scanning/MemoryQuotaSearchTestSystemExtension.java
@@ -34,6 +34,8 @@ import org.junit.jupiter.api.extension.ParameterResolver;
 
 public class MemoryQuotaSearchTestSystemExtension implements ParameterResolver {
 
+    private static final Runnable NO_AWAIT = () -> {};
+
     @Override
     public boolean supportsParameter(ParameterContext parameterContext, ExtensionContext extensionContext) throws ParameterResolutionException {
         return (parameterContext.getParameter().getType() == QuotaSearchTestSystem.class);
@@ -55,7 +57,7 @@ public class MemoryQuotaSearchTestSystemExtension implements ParameterResolver {
                 resources.getMailboxManager(),
                 new ScanningQuotaSearcher(usersRepository,
                     new ClauseConverter(resources.getQuotaRootResolver(), resources.getQuotaManager())),
-                usersRepository, domainList);
+                usersRepository, domainList, NO_AWAIT);
         } catch (Exception e) {
             throw new ParameterResolutionException("Error while resolving parameter", e);
         }

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearchTestSystem.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearchTestSystem.java b/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearchTestSystem.java
index 88e4591..5710a60 100644
--- a/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearchTestSystem.java
+++ b/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearchTestSystem.java
@@ -30,13 +30,15 @@ public class QuotaSearchTestSystem {
     private final QuotaSearcher quotaSearcher;
     private final UsersRepository usersRepository;
     private final DomainList domainList;
+    private final Runnable await;
 
-    public QuotaSearchTestSystem(MaxQuotaManager maxQuotaManager, MailboxManager mailboxManager, QuotaSearcher quotaSearcher, UsersRepository usersRepository, DomainList domainList) {
+    public QuotaSearchTestSystem(MaxQuotaManager maxQuotaManager, MailboxManager mailboxManager, QuotaSearcher quotaSearcher, UsersRepository usersRepository, DomainList domainList, Runnable await) {
         this.maxQuotaManager = maxQuotaManager;
         this.mailboxManager = mailboxManager;
         this.quotaSearcher = quotaSearcher;
         this.usersRepository = usersRepository;
         this.domainList = domainList;
+        this.await = await;
     }
 
     public MaxQuotaManager getMaxQuotaManager() {
@@ -58,4 +60,8 @@ public class QuotaSearchTestSystem {
     public DomainList getDomainList() {
         return domainList;
     }
+
+    public void await() {
+        await.run();
+    }
 }

http://git-wip-us.apache.org/repos/asf/james-project/blob/dd95f7c5/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearcherContract.java
----------------------------------------------------------------------
diff --git a/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearcherContract.java b/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearcherContract.java
index 247d902..0abfc21 100644
--- a/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearcherContract.java
+++ b/mailbox/plugin/quota-search/src/test/java/org/apache/james/quota/search/QuotaSearcherContract.java
@@ -59,6 +59,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.BART, withSize(49));
         appendMessage(testSystem, Simpson.HOMER, withSize(50));
         appendMessage(testSystem, Simpson.LISA, withSize(51));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -78,6 +79,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.BART, withSize(49));
         appendMessage(testSystem, Simpson.HOMER, withSize(50));
         appendMessage(testSystem, Simpson.LISA, withSize(51));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -98,6 +100,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.HOMER, withSize(51));
         appendMessage(testSystem, Simpson.LISA, withSize(60));
         appendMessage(testSystem, BENOIT_AT_DOMAIN_TLD, withSize(80));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -119,6 +122,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.BART, withSize(49));
         appendMessage(testSystem, Simpson.LISA, withSize(51));
         appendMessage(testSystem, BENOIT_AT_DOMAIN_TLD, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -139,6 +143,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.BART, withSize(49));
         appendMessage(testSystem, Simpson.LISA, withSize(51));
         appendMessage(testSystem, BENOIT_AT_DOMAIN_TLD, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -161,6 +166,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Alphabet.ABA, withSize(50));
         appendMessage(testSystem, Alphabet.ACB, withSize(51));
         appendMessage(testSystem, Alphabet.ABB, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -181,6 +187,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Alphabet.ABA, withSize(50));
         appendMessage(testSystem, Alphabet.ACB, withSize(51));
         appendMessage(testSystem, Alphabet.ABB, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -202,6 +209,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Alphabet.ABA, withSize(50));
         appendMessage(testSystem, Alphabet.ACB, withSize(51));
         appendMessage(testSystem, Alphabet.ABB, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -224,6 +232,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Simpson.HOMER, withSize(50));
         appendMessage(testSystem, Simpson.LISA, withSize(51));
         appendMessage(testSystem, BENOIT_AT_DOMAIN_TLD, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()
@@ -245,6 +254,7 @@ public interface QuotaSearcherContract {
         appendMessage(testSystem, Alphabet.ABA, withSize(50));
         appendMessage(testSystem, Alphabet.ACB, withSize(51));
         appendMessage(testSystem, Alphabet.ABB, withSize(50));
+        testSystem.await();
 
         assertThat(
             testSystem.getQuotaSearcher()


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