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 2016/04/22 10:33:16 UTC

[08/13] james-project git commit: JAMES-1717 Provide a Cassandra implementation for vacation storage

JAMES-1717 Provide a Cassandra implementation for vacation storage


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

Branch: refs/heads/master
Commit: d04d1665e8a211687a9f1af48e59e7e78d2265be
Parents: 305dc66
Author: Benoit Tellier <bt...@linagora.com>
Authored: Tue Apr 19 11:30:30 2016 +0700
Committer: Benoit Tellier <bt...@linagora.com>
Committed: Fri Apr 22 15:29:24 2016 +0700

----------------------------------------------------------------------
 .../init/CassandraZonedDateTimeModule.java      |  70 ++++++++++++
 .../utils/ZonedDateTimeRepresentation.java      |  37 +++++++
 .../utils/ZonedDateTimeRepresentationTest.java  |  67 ++++++++++++
 .../modules/mailbox/CassandraSessionModule.java |   5 +
 .../vacation/CassandraVacationDAO.java          | 108 +++++++++++++++++++
 .../vacation/CassandraVacationModule.java       |  71 ++++++++++++
 .../vacation/CassandraVacationRepository.java   |  48 +++++++++
 .../vacation/tables/CassandraVacationTable.java |  31 ++++++
 .../CassandraVacationRepositoryTest.java        |  37 +++++++
 9 files changed, 474 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/init/CassandraZonedDateTimeModule.java
----------------------------------------------------------------------
diff --git a/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/init/CassandraZonedDateTimeModule.java b/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/init/CassandraZonedDateTimeModule.java
new file mode 100644
index 0000000..3a6c949
--- /dev/null
+++ b/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/init/CassandraZonedDateTimeModule.java
@@ -0,0 +1,70 @@
+/****************************************************************
+ * 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.backends.cassandra.init;
+
+import static com.datastax.driver.core.DataType.text;
+import static com.datastax.driver.core.DataType.timestamp;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.james.backends.cassandra.components.CassandraIndex;
+import org.apache.james.backends.cassandra.components.CassandraModule;
+import org.apache.james.backends.cassandra.components.CassandraTable;
+import org.apache.james.backends.cassandra.components.CassandraType;
+
+import com.datastax.driver.core.schemabuilder.SchemaBuilder;
+
+public class CassandraZonedDateTimeModule implements CassandraModule {
+
+    public static final String ZONED_DATE_TIME = "zonedDateTime";
+    public static final String DATE = "date";
+    public static final String TIME_ZONE = "timeZone";
+
+    private final List<CassandraTable> tables;
+    private final List<CassandraIndex> index;
+    private final List<CassandraType> types;
+
+    public CassandraZonedDateTimeModule() {
+        tables = Collections.emptyList();
+        index = Collections.emptyList();
+        types = Collections.singletonList(
+            new CassandraType(ZONED_DATE_TIME,
+                SchemaBuilder.createType(ZONED_DATE_TIME)
+                    .ifNotExists()
+                    .addColumn(DATE, timestamp())
+                    .addColumn(TIME_ZONE, text())));
+    }
+
+    @Override
+    public List<CassandraTable> moduleTables() {
+        return tables;
+    }
+
+    @Override
+    public List<CassandraIndex> moduleIndex() {
+        return index;
+    }
+
+    @Override
+    public List<CassandraType> moduleTypes() {
+        return types;
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentation.java
----------------------------------------------------------------------
diff --git a/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentation.java b/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentation.java
new file mode 100644
index 0000000..f74cc03
--- /dev/null
+++ b/backends-common/cassandra/src/main/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentation.java
@@ -0,0 +1,37 @@
+
+
+package org.apache.james.backends.cassandra.utils;
+
+import java.time.ZoneId;
+import java.time.ZonedDateTime;
+import java.util.Date;
+
+public class ZonedDateTimeRepresentation {
+
+    public static ZonedDateTimeRepresentation fromZonedDateTime(ZonedDateTime zonedDateTime) {
+        return new ZonedDateTimeRepresentation(zonedDateTime);
+    }
+
+    public static ZonedDateTimeRepresentation fromDate(Date date, String serializedZoneId) {
+        return new ZonedDateTimeRepresentation(ZonedDateTime.ofInstant(date.toInstant(), ZoneId.of(serializedZoneId)));
+    }
+
+    private final ZonedDateTime zonedDateTime;
+
+
+    public ZonedDateTimeRepresentation(ZonedDateTime zonedDateTime) {
+        this.zonedDateTime = zonedDateTime;
+    }
+
+    public Date getDate() {
+        return new Date(zonedDateTime.toInstant().toEpochMilli());
+    }
+
+    public String getSerializedZoneId() {
+        return zonedDateTime.getZone().getId();
+    }
+
+    public ZonedDateTime getZonedDateTime() {
+        return zonedDateTime;
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/backends-common/cassandra/src/test/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentationTest.java
----------------------------------------------------------------------
diff --git a/backends-common/cassandra/src/test/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentationTest.java b/backends-common/cassandra/src/test/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentationTest.java
new file mode 100644
index 0000000..be4b973
--- /dev/null
+++ b/backends-common/cassandra/src/test/java/org/apache/james/backends/cassandra/utils/ZonedDateTimeRepresentationTest.java
@@ -0,0 +1,67 @@
+/****************************************************************
+ * 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.backends.cassandra.utils;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+import java.time.ZonedDateTime;
+
+import org.junit.Test;
+
+public class ZonedDateTimeRepresentationTest {
+
+    private static final ZonedDateTime ZONED_DATE_TIME_VN = ZonedDateTime.parse("2016-04-13T12:04:40.906+07:00[Asia/Vientiane]");
+    private static final ZonedDateTime ZONED_DATE_TIME_FR = ZonedDateTime.parse("2016-04-13T07:04:40.906+02:00");
+    private static final long INSTANT = 1460523880906L;
+
+    @Test
+    public void zonedDateTimeRepresentationShouldBeReversible() {
+        ZonedDateTimeRepresentation originalValue = ZonedDateTimeRepresentation.fromZonedDateTime(ZONED_DATE_TIME_VN);
+
+        ZonedDateTimeRepresentation generatedValue = ZonedDateTimeRepresentation.fromDate(originalValue.getDate(), originalValue.getSerializedZoneId());
+
+        assertThat(originalValue.getZonedDateTime()).isEqualTo(generatedValue.getZonedDateTime());
+    }
+
+    @Test
+    public void getSerializedZoneIdShouldReturnTheRightZone() {
+        assertThat(ZonedDateTimeRepresentation.fromZonedDateTime(ZONED_DATE_TIME_VN).getSerializedZoneId())
+            .isEqualTo("Asia/Vientiane");
+    }
+
+    @Test
+    public void getDateShouldReturnTheRightDate() {
+        assertThat(ZonedDateTimeRepresentation.fromZonedDateTime(ZONED_DATE_TIME_VN).getDate().getTime())
+            .isEqualTo(INSTANT);
+    }
+
+    @Test
+    public void getSerializedZoneIdShouldWorkWithFrTimeZone() {
+        assertThat(ZonedDateTimeRepresentation.fromZonedDateTime(ZONED_DATE_TIME_FR).getSerializedZoneId())
+            .isEqualTo("+02:00");
+    }
+
+    @Test
+    public void getDateShouldWorkWithFrTimeZone() {
+        assertThat(ZonedDateTimeRepresentation.fromZonedDateTime(ZONED_DATE_TIME_FR).getDate().getTime())
+            .isEqualTo(INSTANT);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraSessionModule.java
----------------------------------------------------------------------
diff --git a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraSessionModule.java b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraSessionModule.java
index f40cf41..fe234b6 100644
--- a/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraSessionModule.java
+++ b/server/container/guice/cassandra-guice/src/main/java/org/apache/james/modules/mailbox/CassandraSessionModule.java
@@ -27,6 +27,7 @@ import org.apache.commons.configuration.ConfigurationException;
 import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.james.backends.cassandra.components.CassandraModule;
 import org.apache.james.backends.cassandra.init.CassandraModuleComposite;
+import org.apache.james.backends.cassandra.init.CassandraZonedDateTimeModule;
 import org.apache.james.backends.cassandra.init.ClusterFactory;
 import org.apache.james.backends.cassandra.init.ClusterWithKeyspaceCreatedFactory;
 import org.apache.james.backends.cassandra.init.SessionWithInitializedTablesFactory;
@@ -38,6 +39,7 @@ import com.datastax.driver.core.exceptions.NoHostAvailableException;
 import com.google.inject.AbstractModule;
 import com.google.inject.Provides;
 import com.google.inject.Singleton;
+import com.google.inject.multibindings.Multibinder;
 import com.nurkiewicz.asyncretry.AsyncRetryExecutor;
 
 public class CassandraSessionModule extends AbstractModule {
@@ -48,6 +50,9 @@ public class CassandraSessionModule extends AbstractModule {
     @Override
     protected void configure() {
         bind(ScheduledExecutorService.class).toProvider(ScheduledExecutorServiceProvider.class);
+
+        Multibinder<CassandraModule> cassandraDataDefinitions = Multibinder.newSetBinder(binder(), CassandraModule.class);
+        cassandraDataDefinitions.addBinding().to(CassandraZonedDateTimeModule.class);
     }
     
     @Provides

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationDAO.java
----------------------------------------------------------------------
diff --git a/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationDAO.java b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationDAO.java
new file mode 100644
index 0000000..cc1c647
--- /dev/null
+++ b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationDAO.java
@@ -0,0 +1,108 @@
+/****************************************************************
+ * 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.jmap.cassandra.vacation;
+
+import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.eq;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.insertInto;
+import static com.datastax.driver.core.querybuilder.QueryBuilder.select;
+
+import java.time.ZonedDateTime;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import javax.inject.Inject;
+
+import org.apache.james.backends.cassandra.init.CassandraTypesProvider;
+import org.apache.james.backends.cassandra.init.CassandraZonedDateTimeModule;
+import org.apache.james.backends.cassandra.utils.CassandraAsyncExecutor;
+import org.apache.james.backends.cassandra.utils.ZonedDateTimeRepresentation;
+import org.apache.james.jmap.api.vacation.AccountId;
+import org.apache.james.jmap.api.vacation.Vacation;
+import org.apache.james.jmap.cassandra.vacation.tables.CassandraVacationTable;
+
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.UDTValue;
+import com.datastax.driver.core.UserType;
+
+public class CassandraVacationDAO {
+
+    private final CassandraAsyncExecutor cassandraAsyncExecutor;
+    private final PreparedStatement insertStatement;
+    private final PreparedStatement readStatement;
+    private final UserType zonedDateTimeUserType;
+
+    @Inject
+    public CassandraVacationDAO(Session session, CassandraTypesProvider cassandraTypesProvider) {
+        this.zonedDateTimeUserType = cassandraTypesProvider.getDefinedUserType(CassandraZonedDateTimeModule.ZONED_DATE_TIME);
+        this.cassandraAsyncExecutor = new CassandraAsyncExecutor(session);
+
+        this.insertStatement = session.prepare(insertInto(CassandraVacationTable.TABLE_NAME)
+            .value(CassandraVacationTable.ACCOUNT_ID, bindMarker(CassandraVacationTable.ACCOUNT_ID))
+            .value(CassandraVacationTable.IS_ENABLED, bindMarker(CassandraVacationTable.IS_ENABLED))
+            .value(CassandraVacationTable.FROM_DATE, bindMarker(CassandraVacationTable.FROM_DATE))
+            .value(CassandraVacationTable.TO_DATE, bindMarker(CassandraVacationTable.TO_DATE))
+            .value(CassandraVacationTable.TEXT, bindMarker(CassandraVacationTable.TEXT)));
+
+        this.readStatement = session.prepare(select()
+            .from(CassandraVacationTable.TABLE_NAME)
+            .where(eq(CassandraVacationTable.ACCOUNT_ID,
+                bindMarker(CassandraVacationTable.ACCOUNT_ID))));
+    }
+
+    public CompletableFuture<Void> modifyVacation(AccountId accountId, Vacation vacation) {
+        return cassandraAsyncExecutor.executeVoid(
+            insertStatement.bind()
+                .setString(CassandraVacationTable.ACCOUNT_ID, accountId.getIdentifier())
+                .setBool(CassandraVacationTable.IS_ENABLED, vacation.isEnabled())
+                .setUDTValue(CassandraVacationTable.FROM_DATE, convertToUDTValue(vacation.getFromDate()))
+                .setUDTValue(CassandraVacationTable.TO_DATE, convertToUDTValue(vacation.getToDate()))
+                .setString(CassandraVacationTable.TEXT, vacation.getTextBody()));
+    }
+
+    public CompletableFuture<Optional<Vacation>> retrieveVacation(AccountId accountId) {
+        return cassandraAsyncExecutor.executeSingleRow(readStatement.bind()
+            .setString(CassandraVacationTable.ACCOUNT_ID, accountId.getIdentifier()))
+            .thenApply(optional -> optional.map(row -> Vacation.builder()
+                .enabled(row.getBool(CassandraVacationTable.IS_ENABLED))
+                .fromDate(retrieveDate(row, CassandraVacationTable.FROM_DATE))
+                .toDate(retrieveDate(row, CassandraVacationTable.TO_DATE))
+                .textBody(row.getString(CassandraVacationTable.TEXT))
+                .build()));
+    }
+
+    private Optional<ZonedDateTime> retrieveDate(Row row, String dateField) {
+        return Optional.ofNullable(row.getUDTValue(dateField))
+            .map(udtValue -> ZonedDateTimeRepresentation.fromDate(
+                udtValue.getDate(CassandraZonedDateTimeModule.DATE),
+                udtValue.getString(CassandraZonedDateTimeModule.TIME_ZONE))
+                .getZonedDateTime());
+    }
+
+    private UDTValue convertToUDTValue(Optional<ZonedDateTime> zonedDateTimeOptional) {
+        return zonedDateTimeOptional.map(ZonedDateTimeRepresentation::fromZonedDateTime)
+            .map(representation -> zonedDateTimeUserType.newValue()
+                .setDate(CassandraZonedDateTimeModule.DATE, representation.getDate())
+                .setString(CassandraZonedDateTimeModule.TIME_ZONE, representation.getSerializedZoneId()))
+            .orElse(null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationModule.java
----------------------------------------------------------------------
diff --git a/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationModule.java b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationModule.java
new file mode 100644
index 0000000..dae2451
--- /dev/null
+++ b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationModule.java
@@ -0,0 +1,71 @@
+/****************************************************************
+ * 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.jmap.cassandra.vacation;
+
+import static com.datastax.driver.core.DataType.cboolean;
+import static com.datastax.driver.core.DataType.text;
+
+import java.util.List;
+
+import org.apache.james.backends.cassandra.components.CassandraIndex;
+import org.apache.james.backends.cassandra.components.CassandraModule;
+import org.apache.james.backends.cassandra.components.CassandraTable;
+import org.apache.james.backends.cassandra.components.CassandraType;
+import org.apache.james.backends.cassandra.init.CassandraZonedDateTimeModule;
+import org.apache.james.jmap.cassandra.vacation.tables.CassandraVacationTable;
+
+import com.datastax.driver.core.schemabuilder.SchemaBuilder;
+import com.google.common.collect.ImmutableList;
+
+public class CassandraVacationModule implements CassandraModule {
+
+    private final List<CassandraTable> tables;
+    private final List<CassandraIndex> index;
+    private final List<CassandraType> types;
+
+    public CassandraVacationModule() {
+        tables = ImmutableList.of(
+            new CassandraTable(CassandraVacationTable.TABLE_NAME,
+                SchemaBuilder.createTable(CassandraVacationTable.TABLE_NAME)
+                    .ifNotExists()
+                    .addPartitionKey(CassandraVacationTable.ACCOUNT_ID, text())
+                    .addColumn(CassandraVacationTable.IS_ENABLED, cboolean())
+                    .addUDTColumn(CassandraVacationTable.FROM_DATE, SchemaBuilder.frozen(CassandraZonedDateTimeModule.ZONED_DATE_TIME))
+                    .addUDTColumn(CassandraVacationTable.TO_DATE, SchemaBuilder.frozen(CassandraZonedDateTimeModule.ZONED_DATE_TIME))
+                    .addColumn(CassandraVacationTable.TEXT, text())));
+        index = ImmutableList.of();
+        types = ImmutableList.of();
+    }
+
+    @Override
+    public List<CassandraTable> moduleTables() {
+        return tables;
+    }
+
+    @Override
+    public List<CassandraIndex> moduleIndex() {
+        return index;
+    }
+
+    @Override
+    public List<CassandraType> moduleTypes() {
+        return types;
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepository.java
----------------------------------------------------------------------
diff --git a/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepository.java b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepository.java
new file mode 100644
index 0000000..60958d4
--- /dev/null
+++ b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepository.java
@@ -0,0 +1,48 @@
+/****************************************************************
+ * 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.jmap.cassandra.vacation;
+
+import java.util.concurrent.CompletableFuture;
+
+import javax.inject.Inject;
+
+import org.apache.james.jmap.api.vacation.AccountId;
+import org.apache.james.jmap.api.vacation.Vacation;
+import org.apache.james.jmap.api.vacation.VacationRepository;
+
+public class CassandraVacationRepository implements VacationRepository {
+
+    private final CassandraVacationDAO cassandraVacationDAO;
+
+    @Inject
+    public CassandraVacationRepository(CassandraVacationDAO cassandraVacationDAO) {
+        this.cassandraVacationDAO = cassandraVacationDAO;
+    }
+
+    @Override
+    public CompletableFuture<Void> modifyVacation(AccountId accountId, Vacation vacation) {
+        return cassandraVacationDAO.modifyVacation(accountId, vacation);
+    }
+
+    @Override
+    public CompletableFuture<Vacation> retrieveVacation(AccountId accountId) {
+        return cassandraVacationDAO.retrieveVacation(accountId).thenApply(optional -> optional.orElse(DEFAULT_VACATION));
+    }
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/tables/CassandraVacationTable.java
----------------------------------------------------------------------
diff --git a/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/tables/CassandraVacationTable.java b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/tables/CassandraVacationTable.java
new file mode 100644
index 0000000..7e9b3e2
--- /dev/null
+++ b/server/data/data-jmap-cassandra/src/main/java/org/apache/james/jmap/cassandra/vacation/tables/CassandraVacationTable.java
@@ -0,0 +1,31 @@
+/****************************************************************
+ * 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.jmap.cassandra.vacation.tables;
+
+public interface CassandraVacationTable {
+
+    String TABLE_NAME = "vacation";
+    String ACCOUNT_ID = "account_id";
+    String FROM_DATE = "from_date";
+    String TO_DATE = "to_date";
+    String IS_ENABLED = "is_enabled";
+    String TEXT = "text";
+
+}

http://git-wip-us.apache.org/repos/asf/james-project/blob/d04d1665/server/data/data-jmap-cassandra/src/test/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepositoryTest.java
----------------------------------------------------------------------
diff --git a/server/data/data-jmap-cassandra/src/test/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepositoryTest.java b/server/data/data-jmap-cassandra/src/test/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepositoryTest.java
new file mode 100644
index 0000000..d0918a9
--- /dev/null
+++ b/server/data/data-jmap-cassandra/src/test/java/org/apache/james/jmap/cassandra/vacation/CassandraVacationRepositoryTest.java
@@ -0,0 +1,37 @@
+/****************************************************************
+ * 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.jmap.cassandra.vacation;
+
+import org.apache.james.backends.cassandra.CassandraCluster;
+import org.apache.james.backends.cassandra.init.CassandraModuleComposite;
+import org.apache.james.backends.cassandra.init.CassandraZonedDateTimeModule;
+import org.apache.james.jmap.api.vacation.AbstractVacationRepositoryTest;
+import org.apache.james.jmap.api.vacation.VacationRepository;
+
+public class CassandraVacationRepositoryTest extends AbstractVacationRepositoryTest {
+
+    private CassandraCluster cassandra;
+
+    @Override
+    protected VacationRepository createVacationRepository() {
+        cassandra = CassandraCluster.create(new CassandraModuleComposite(new CassandraVacationModule(), new CassandraZonedDateTimeModule()));
+        return new CassandraVacationRepository(new CassandraVacationDAO(cassandra.getConf(), cassandra.getTypesProvider()));
+    }
+}


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