[jira] [Commented] (JAMES-2808) DeletedMessage serialization & deserialization
[ https://issues.apache.org/jira/browse/JAMES-2808?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16880928#comment-16880928 ] Rene Cordier commented on JAMES-2808: - PR : https://github.com/linagora/james-project/pull/2502 > DeletedMessage serialization & deserialization > -- > > Key: JAMES-2808 > URL: https://issues.apache.org/jira/browse/JAMES-2808 > Project: James Server > Issue Type: Sub-task > Components: deletedMessageVault >Reporter: Tellier Benoit >Priority: Major > > BlobStore takes/retrieves InputStream/bytes. We need a component to > serialize/deserialize DeletedMessage and related storage information into > InputStream and vice versa. (See `DeletedMessageWithStorageInformation` in > JAMES-2807 ) > Currently, there are two parts of a deleted message: > - Metadata represented by the DTO: > {code:java} > public class DeletedMessage { > private final MessageId messageId; > private final List originMailboxes; > private final User owner; > private final ZonedDateTime deliveryDate; > private final ZonedDateTime deletionDate; > private final MaybeSender sender; > private final List recipients; > private final Optional subject; > private final boolean hasAttachment; > private final long size; > // storage information > private final BucketName bucketName; > private final BlobId blobId; > } > {code} > - Content represtented by InputStream > We can consider the Serializer will be in charge of serializing/deserializing > just DTOs. And the Vault saves each deleted message into two parts, metadata > will be stored in DeletedMessageMetadataVault. and It uses this serializer to > serialize DeletedMessage into json and save it as a field in cassandra column -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Commented] (JAMES-2806) Adding bucket capability into BlobStore
[ https://issues.apache.org/jira/browse/JAMES-2806?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16880495#comment-16880495 ] Tellier Benoit commented on JAMES-2806: --- https://github.com/linagora/james-project/pull/2492 contributed the object storage implementation of this > Adding bucket capability into BlobStore > --- > > Key: JAMES-2806 > URL: https://issues.apache.org/jira/browse/JAMES-2806 > Project: James Server > Issue Type: Sub-task > Components: Blob >Reporter: Tellier Benoit >Priority: Major > Fix For: 3.4.0 > > > You should update current BlobStore API to be able to customize the way we > storeBlob: > - We want to store a blob in a specified bucket > {code:java} > public interface BlobStore { > Publisher save(BucketName, byte[] data); > Publisher save(BucketName, InputStream data); > Publisher readBytes(BucketName, BlobId blobId); > InputStream read(BucketName, BlobId blobId); > } > {code} > And to implement the feature of deleting buckets exceed retention time, we > need to have another additional method of deleting bucket: > {code:java} > public interface BlobStore { > Publisher delete(BucketName bucket); > } > {code} > Scope: API change, contract test, memory + object storage implementation - > cassandra will be done in a later task -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] 06/07: JAMES-2806 ObjectStorage Swift retry one more time when fails to save
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 2232634235d72a8c4ed0a84a85422118fe662bbd Author: Tran Tien Duc AuthorDate: Fri Jul 5 17:37:39 2019 +0700 JAMES-2806 ObjectStorage Swift retry one more time when fails to save Swift cause a big time wait for the first put blob request to response. Assuming there are 2 thread trying to save a blob in the same time, and both are expected to fail and when a thread fall back to retry and create the bucket, the other thread comes to the step of checking bucket existing, now the bucket is already created then retry mechanism is rejected. I want to have the last retry regardless bucket existence to increase the chance to save a blob. This is just a conner case and rarely we never come up with retrying saving blob. Except the only one use case of Vault switching new Bucket to save deleted message. Then I have to take care about this issue rather than relax test concurrency level or disable it. --- .../objectstorage/StreamCompatibleBlobPutter.java | 24 ++ 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java index 9a3a21f..2375864 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java @@ -42,6 +42,7 @@ public class StreamCompatibleBlobPutter implements BlobPutter { private static final Duration FIRST_BACK_OFF = Duration.ofMillis(100); private static final Duration FOREVER = Duration.ofMillis(Long.MAX_VALUE); private static final Location DEFAULT_LOCATION = null; +private static final long RETRY_ONE_LAST_TIME_ON_CONCURRENT_SAVING = 1; private final BlobStore blobStore; @@ -58,6 +59,10 @@ public class StreamCompatibleBlobPutter implements BlobPutter { .withBackoffScheduler(Schedulers.elastic()) .retryMax(MAX_RETRIES) .doOnRetry(retryContext -> blobStore.createContainerInLocation(DEFAULT_LOCATION, bucketName.asString( +.retryWhen(Retry.onlyIf(RetryContext -> isPutMethod(RetryContext.exception())) +.withBackoffScheduler(Schedulers.elastic()) +.exponentialBackoff(FIRST_BACK_OFF, FOREVER) +.retryMax(RETRY_ONE_LAST_TIME_ON_CONCURRENT_SAVING)) .block(); } @@ -76,15 +81,16 @@ public class StreamCompatibleBlobPutter implements BlobPutter { } private boolean needToCreateBucket(Throwable throwable, BucketName bucketName) { -if (throwable instanceof HttpResponseException -|| throwable instanceof KeyNotFoundException) { - -return extractHttpException(throwable) -.map(ex -> isPutMethod(ex) && !bucketExisted(bucketName)) -.orElse(false); -} +return Optional.of(throwable) +.filter(t -> t instanceof HttpResponseException || t instanceof KeyNotFoundException) +.flatMap(this::extractHttpException) +.map(ex -> isPutMethod(ex) && !bucketExists(bucketName)) +.orElse(false); +} -return false; +private boolean isPutMethod(Throwable throwable) { +return throwable instanceof HttpResponseException +&& isPutMethod((HttpResponseException) throwable); } private boolean isPutMethod(HttpResponseException ex) { @@ -94,7 +100,7 @@ public class StreamCompatibleBlobPutter implements BlobPutter { .equals("PUT"); } -private boolean bucketExisted(BucketName bucketName) { +private boolean bucketExists(BucketName bucketName) { return blobStore.containerExists(bucketName.asString()); } - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] 02/07: JAMES-2806 ObjectStorage Swift save retry save() on non existing bucket
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 542b4cbf4fdc7aac7c5d59e8879fea3063e2c963 Author: Tran Tien Duc AuthorDate: Thu Jul 4 17:09:03 2019 +0700 JAMES-2806 ObjectStorage Swift save retry save() on non existing bucket --- server/blob/blob-objectstorage/pom.xml | 4 +++ .../objectstorage/StreamCompatibleBlobPutter.java | 34 -- .../objectstorage/ObjectStorageBlobsDAOTest.java | 12 +++- 3 files changed, 40 insertions(+), 10 deletions(-) diff --git a/server/blob/blob-objectstorage/pom.xml b/server/blob/blob-objectstorage/pom.xml index c312e93..dabae94 100644 --- a/server/blob/blob-objectstorage/pom.xml +++ b/server/blob/blob-objectstorage/pom.xml @@ -87,6 +87,10 @@ guava +io.projectreactor.addons +reactor-extra + + nl.jqno.equalsverifier equalsverifier test diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java index f01bc34..1bac8e2 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java @@ -19,6 +19,7 @@ package org.apache.james.blob.objectstorage; +import java.time.Duration; import java.util.function.Supplier; import org.apache.james.blob.api.BlobId; @@ -26,9 +27,21 @@ import org.apache.james.blob.api.BucketName; import org.jclouds.blobstore.BlobStore; import org.jclouds.blobstore.domain.Blob; import org.jclouds.blobstore.options.CopyOptions; +import org.jclouds.domain.Location; +import org.jclouds.http.HttpResponseException; + +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; +import reactor.retry.Retry; public class StreamCompatibleBlobPutter implements BlobPutter { -private final org.jclouds.blobstore.BlobStore blobStore; + +private static final int MAX_RETRIES = 3; +private static final Duration FIRST_BACK_OFF = Duration.ofMillis(100); +private static final Duration FOREVER = Duration.ofMillis(Long.MAX_VALUE); +private static final Location DEFAULT_LOCATION = null; + +private final BlobStore blobStore; public StreamCompatibleBlobPutter(BlobStore blobStore) { this.blobStore = blobStore; @@ -36,7 +49,14 @@ public class StreamCompatibleBlobPutter implements BlobPutter { @Override public void putDirectly(BucketName bucketName, Blob blob) { -blobStore.putBlob(bucketName.asString(), blob); +Mono.fromRunnable(() -> blobStore.putBlob(bucketName.asString(), blob)) +.publishOn(Schedulers.elastic()) +.retryWhen(Retry.onlyIf(retryContext -> needToCreateBucket(retryContext.exception(), bucketName)) +.exponentialBackoff(FIRST_BACK_OFF, FOREVER) +.withBackoffScheduler(Schedulers.elastic()) +.retryMax(MAX_RETRIES) +.doOnRetry(retryContext -> blobStore.createContainerInLocation(DEFAULT_LOCATION, bucketName.asString( +.block(); } @Override @@ -52,4 +72,14 @@ public class StreamCompatibleBlobPutter implements BlobPutter { blobStore.copyBlob(bucketNameAsString, from, bucketNameAsString, to, CopyOptions.NONE); blobStore.removeBlob(bucketNameAsString, from); } + +private boolean needToCreateBucket(Throwable throwable, BucketName bucketName) { +if (throwable instanceof HttpResponseException) { +HttpResponseException ex = (HttpResponseException) throwable; +return ex.getCommand().getCurrentRequest().getMethod().equals("PUT") +&& !blobStore.containerExists(bucketName.asString()); +} + +return false; +} } \ No newline at end of file diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java index 12f11b8..a6321f7 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java @@ -31,6 +31,7 @@ import java.util.UUID; import org.apache.commons.io.IOUtils; import org.apache.james.blob.api.BlobId; import org.apache.james.blob.api.BlobStore; +import org.apache.james.blob.api.BucketBlobStoreContract; import org.apache.james.blob.api.BucketName; import
[james-project] 04/07: JAMES-2806 ObjectStorageBlobDAO no need to create bucket explicitly
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 126320bbb2db3836a6700905b9f3e2deb1c95bf6 Author: Tran Tien Duc AuthorDate: Fri Jul 5 12:59:51 2019 +0700 JAMES-2806 ObjectStorageBlobDAO no need to create bucket explicitly --- .../blob/objectstorage/ObjectStorageBlobsDAO.java | 7 --- .../objectstorage/ObjectStorageBlobsDAOAWSTest.java| 1 - .../objectstorage/ObjectStorageBlobsDAOContract.java | 1 - .../blob/objectstorage/ObjectStorageBlobsDAOTest.java | 18 -- .../objectstorage/ObjectStorageDependenciesModule.java | 2 -- .../swift/ObjectStorageBlobStoreModuleTest.java| 4 6 files changed, 33 deletions(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java index 94f63fe..3a0bf83 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java @@ -86,13 +86,6 @@ public class ObjectStorageBlobsDAO implements BlobStore { return AwsS3ObjectStorage.daoBuilder(testConfig); } -public Mono createBucket(BucketName name) { -return Mono.fromCallable(() -> blobStore.createContainerInLocation(DEFAULT_LOCATION, name.asString())) -.filter(created -> created == false) -.doOnNext(ignored -> LOGGER.debug("{} already existed", name)) -.thenReturn(name); -} - @Override public Mono save(BucketName bucketName, byte[] data) { Preconditions.checkNotNull(data); diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java index 716ff23..b9f101b 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java @@ -66,7 +66,6 @@ public class ObjectStorageBlobsDAOAWSTest implements MetricableBlobStoreContract blobStore = builder.getSupplier().get(); objectStorageBlobsDAO = builder.build(); -objectStorageBlobsDAO.createBucket(defaultBucketName).block(); testee = new MetricableBlobStore(metricsTestExtension.getMetricFactory(), objectStorageBlobsDAO); } diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOContract.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOContract.java index f25393a..92bf2f8 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOContract.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOContract.java @@ -37,7 +37,6 @@ public interface ObjectStorageBlobsDAOContract { default void assertBlobsDAOCanStoreAndRetrieve(ObjectStorageBlobsDAOBuilder.ReadyToBuild builder) { ObjectStorageBlobsDAO dao = builder.build(); -dao.createBucket(dao.getDefaultBucketName()).block(); BlobId blobId = dao.save(dao.getDefaultBucketName(), CONTENT).block(); diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java index a6321f7..9423ee8 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java @@ -20,7 +20,6 @@ package org.apache.james.blob.objectstorage; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatCode; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -92,7 +91,6 @@ public class ObjectStorageBlobsDAOTest implements MetricableBlobStoreContract, B .blobIdFactory(blobIdFactory); blobStore = daoBuilder.getSupplier().get(); objectStorageBlobsDAO = daoBuilder.build(); -objectStorageBlobsDAO.createBucket(defaultBucketName).block(); testee = new MetricableBlobStore(metricsTestExtension.getMetricFactory(), objectStorageBlobsDAO); } @@ -114,22 +112,6 @@ public class ObjectStorageBlobsDAOTest implements
[james-project] 01/07: JAMES-2806 Upgrade Jcloud version
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 6f984a184bc10408d72017e20b61ddde35de1757 Author: Tran Tien Duc AuthorDate: Tue Jul 2 17:28:40 2019 +0700 JAMES-2806 Upgrade Jcloud version This upgrade contains a fix for BlobStore Container cache validation that the following commit need it to not break the ObjectStorage DAO with Swift --- server/blob/blob-objectstorage/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/blob/blob-objectstorage/pom.xml b/server/blob/blob-objectstorage/pom.xml index a92e073..c312e93 100644 --- a/server/blob/blob-objectstorage/pom.xml +++ b/server/blob/blob-objectstorage/pom.xml @@ -33,7 +33,7 @@ Apache James :: Server :: Blob :: Object storage -2.1.1 +2.1.2 - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] 05/07: JAMES-2806 ObjectStorage Swift should retry when saving empty blob
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 95456efc99f2b4e3ac737ed71aa698dbd8a6b8bb Author: Tran Tien Duc AuthorDate: Fri Jul 5 17:16:57 2019 +0700 JAMES-2806 ObjectStorage Swift should retry when saving empty blob The KeyNotFound exception thrown when saving empty blob in non existing bucket, we need to catch it also with the HttpResponseException --- .../objectstorage/StreamCompatibleBlobPutter.java | 33 +++--- 1 file changed, 29 insertions(+), 4 deletions(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java index 1bac8e2..9a3a21f 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java @@ -20,11 +20,13 @@ package org.apache.james.blob.objectstorage; import java.time.Duration; +import java.util.Optional; import java.util.function.Supplier; import org.apache.james.blob.api.BlobId; import org.apache.james.blob.api.BucketName; import org.jclouds.blobstore.BlobStore; +import org.jclouds.blobstore.KeyNotFoundException; import org.jclouds.blobstore.domain.Blob; import org.jclouds.blobstore.options.CopyOptions; import org.jclouds.domain.Location; @@ -74,12 +76,35 @@ public class StreamCompatibleBlobPutter implements BlobPutter { } private boolean needToCreateBucket(Throwable throwable, BucketName bucketName) { -if (throwable instanceof HttpResponseException) { -HttpResponseException ex = (HttpResponseException) throwable; -return ex.getCommand().getCurrentRequest().getMethod().equals("PUT") -&& !blobStore.containerExists(bucketName.asString()); +if (throwable instanceof HttpResponseException +|| throwable instanceof KeyNotFoundException) { + +return extractHttpException(throwable) +.map(ex -> isPutMethod(ex) && !bucketExisted(bucketName)) +.orElse(false); } return false; } + +private boolean isPutMethod(HttpResponseException ex) { +return ex.getCommand() +.getCurrentRequest() +.getMethod() +.equals("PUT"); +} + +private boolean bucketExisted(BucketName bucketName) { +return blobStore.containerExists(bucketName.asString()); +} + +private Optional extractHttpException(Throwable throwable) { +if (throwable instanceof HttpResponseException) { +return Optional.of((HttpResponseException) throwable); +} else if (throwable.getCause() instanceof HttpResponseException) { +return Optional.of((HttpResponseException) throwable.getCause()); +} + +return Optional.empty(); +} } \ No newline at end of file - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] 07/07: JAMES-2806 use fixed bucket name in tests
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit ceb1d9e71d5b6fa77b1c16e4fe4ddcabb35b Author: Tran Tien Duc AuthorDate: Mon Jul 8 10:09:17 2019 +0700 JAMES-2806 use fixed bucket name in tests --- .../apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java | 4 +--- .../apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java| 3 +-- .../blob/objectstorage/aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java | 4 +--- .../swift/SwiftKeystone2ObjectStorageBlobsDAOBuilderTest.java | 3 +-- .../swift/SwiftKeystone3ObjectStorageBlobsDAOBuilderTest.java | 3 +-- .../swift/SwiftTempAuthObjectStorageBlobsDAOBuilderTest.java | 3 +-- 6 files changed, 6 insertions(+), 14 deletions(-) diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java index b9f101b..209be69 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java @@ -19,8 +19,6 @@ package org.apache.james.blob.objectstorage; -import java.util.UUID; - import org.apache.james.blob.api.BlobId; import org.apache.james.blob.api.BlobStore; import org.apache.james.blob.api.BucketBlobStoreContract; @@ -51,7 +49,7 @@ public class ObjectStorageBlobsDAOAWSTest implements MetricableBlobStoreContract @BeforeEach void setUp(DockerAwsS3Container dockerAwsS3) { awsS3ObjectStorage = new AwsS3ObjectStorage(); -defaultBucketName = BucketName.of(UUID.randomUUID().toString()); +defaultBucketName = BucketName.of("e7a929fe-2003-48d4-aaa9-a65d4954379d"); configuration = AwsS3AuthConfiguration.builder() .endpoint(dockerAwsS3.getEndpoint()) .accessKeyId(DockerAwsS3Container.ACCESS_KEY_ID) diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java index 9423ee8..18c6bf2 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOTest.java @@ -25,7 +25,6 @@ import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; -import java.util.UUID; import org.apache.commons.io.IOUtils; import org.apache.james.blob.api.BlobId; @@ -76,7 +75,7 @@ public class ObjectStorageBlobsDAOTest implements MetricableBlobStoreContract, B @BeforeEach void setUp(DockerSwift dockerSwift) { -defaultBucketName = BucketName.of(UUID.randomUUID().toString()); +defaultBucketName = BucketName.of("e4fc2427-f2aa-422a-a535-3df0d2a086c4"); testConfig = SwiftTempAuthObjectStorage.configBuilder() .endpoint(dockerSwift.swiftEndpoint()) .identity(SWIFT_IDENTITY) diff --git a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java index 1592c99..a81e930 100644 --- a/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java +++ b/server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java @@ -21,8 +21,6 @@ package org.apache.james.blob.objectstorage.aws; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.util.UUID; - import org.apache.james.blob.api.BucketName; import org.apache.james.blob.api.HashBlobId; import org.apache.james.blob.objectstorage.ObjectStorageBlobsDAO; @@ -43,7 +41,7 @@ class AwsS3ObjectStorageBlobsDAOBuilderTest implements ObjectStorageBlobsDAOCont @BeforeEach void setUp(DockerAwsS3Container dockerAwsS3Container) { awsS3ObjectStorage = new AwsS3ObjectStorage(); -defaultBucketName = BucketName.of(UUID.randomUUID().toString()); +defaultBucketName = BucketName.of("d1953ef8-cfe8-460b-bc29-3977f5b6656f"); configuration = AwsS3AuthConfiguration.builder() .endpoint(dockerAwsS3Container.getEndpoint()) .accessKeyId(DockerAwsS3Container.ACCESS_KEY_ID) diff --git
[james-project] 03/07: JAMES-2806 ObjectStorage S3 retry save() on non existing bucket
This is an automated email from the ASF dual-hosted git repository. btellier pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit b5fb2f7174e177944336953afa4fb9e09f615c3e Author: Tran Tien Duc AuthorDate: Thu Jul 4 17:09:51 2019 +0700 JAMES-2806 ObjectStorage S3 retry save() on non existing bucket --- .../blob/objectstorage/aws/AwsS3ObjectStorage.java | 72 +++-- .../ObjectStorageBlobsDAOAWSTest.java | 90 ++ 2 files changed, 137 insertions(+), 25 deletions(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorage.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorage.java index 1b52c85..4884dd5 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorage.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/aws/AwsS3ObjectStorage.java @@ -21,6 +21,7 @@ package org.apache.james.blob.objectstorage.aws; import java.io.File; import java.io.IOException; +import java.time.Duration; import java.util.Optional; import java.util.Properties; import java.util.UUID; @@ -45,7 +46,6 @@ import org.jclouds.blobstore.BlobStoreContext; import org.jclouds.blobstore.domain.Blob; import org.jclouds.logging.slf4j.config.SLF4JLoggingModule; -import com.amazonaws.AmazonClientException; import com.amazonaws.ClientConfiguration; import com.amazonaws.auth.AWSStaticCredentialsProvider; import com.amazonaws.auth.BasicAWSCredentials; @@ -53,13 +53,19 @@ import com.amazonaws.client.builder.AwsClientBuilder; import com.amazonaws.retry.PredefinedRetryPolicies; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3ClientBuilder; +import com.amazonaws.services.s3.model.AmazonS3Exception; import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.transfer.TransferManager; import com.amazonaws.services.s3.transfer.TransferManagerBuilder; +import com.github.fge.lambdas.Throwing; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; import com.google.inject.Module; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; +import reactor.retry.Retry; + public class AwsS3ObjectStorage { private static final Iterable JCLOUDS_MODULES = ImmutableSet.of(new SLF4JLoggingModule()); @@ -82,7 +88,7 @@ public class AwsS3ObjectStorage { @Inject @VisibleForTesting -AwsS3ObjectStorage() { +public AwsS3ObjectStorage() { executorService = Executors.newFixedThreadPool(MAX_THREADS, NamedThreadFactory.withClassName(AwsS3ObjectStorage.class)); } @@ -125,6 +131,12 @@ public class AwsS3ObjectStorage { } private static class AwsS3BlobPutter implements BlobPutter { + +private static final int NOT_FOUND_STATUS_CODE = 404; +private static final String BUCKET_NOT_FOUND_ERROR_CODE = "NoSuchBucket"; +private static final Duration FIRST_BACK_OFF = Duration.ofMillis(100); +private static final Duration FOREVER = Duration.ofMillis(Long.MAX_VALUE); + private final AwsS3AuthConfiguration configuration; private final ExecutorService executorService; @@ -135,14 +147,14 @@ public class AwsS3ObjectStorage { @Override public void putDirectly(BucketName bucketName, Blob blob) { -writeFileAndAct(blob, (file) -> putWithRetry(bucketName, configuration, blob, file, FIRST_TRY)); +writeFileAndAct(blob, (file) -> putWithRetry(bucketName, configuration, blob, file, FIRST_TRY).block()); } @Override public BlobId putAndComputeId(BucketName bucketName, Blob initialBlob, Supplier blobIdSupplier) { Consumer putChangedBlob = (file) -> { initialBlob.getMetadata().setName(blobIdSupplier.get().asString()); -putWithRetry(bucketName, configuration, initialBlob, file, FIRST_TRY); +putWithRetry(bucketName, configuration, initialBlob, file, FIRST_TRY).block(); }; writeFileAndAct(initialBlob, putChangedBlob); return blobIdSupplier.get(); @@ -163,30 +175,40 @@ public class AwsS3ObjectStorage { } } -private void putWithRetry(BucketName bucketName, AwsS3AuthConfiguration configuration, Blob blob, File file, int tried) { -try { -put(bucketName, configuration, blob, file); -} catch (RuntimeException e) { -if (tried < MAX_RETRY_ON_EXCEPTION) { -putWithRetry(bucketName, configuration, blob, file, tried + 1); -} else { -throw e; -} -} +private Mono putWithRetry(BucketName
[james-project] branch master updated (92da31b -> c6666eb)
This is an automated email from the ASF dual-hosted git repository. btellier pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git. from 92da31b JAMES-2815 Store directly small blobs in ObjectStore new 6f984a1 JAMES-2806 Upgrade Jcloud version new 542b4cb JAMES-2806 ObjectStorage Swift save retry save() on non existing bucket new b5fb2f7 JAMES-2806 ObjectStorage S3 retry save() on non existing bucket new 126320b JAMES-2806 ObjectStorageBlobDAO no need to create bucket explicitly new 95456ef JAMES-2806 ObjectStorage Swift should retry when saving empty blob new 2232634 JAMES-2806 ObjectStorage Swift retry one more time when fails to save new ceb JAMES-2806 use fixed bucket name in tests The 7 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: server/blob/blob-objectstorage/pom.xml | 6 +- .../blob/objectstorage/ObjectStorageBlobsDAO.java | 7 -- .../objectstorage/StreamCompatibleBlobPutter.java | 65 +++- .../blob/objectstorage/aws/AwsS3ObjectStorage.java | 72 +++--- .../ObjectStorageBlobsDAOAWSTest.java | 87 ++ .../ObjectStorageBlobsDAOContract.java | 1 - .../objectstorage/ObjectStorageBlobsDAOTest.java | 33 ++-- .../aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java | 4 +- ...tKeystone2ObjectStorageBlobsDAOBuilderTest.java | 3 +- ...tKeystone3ObjectStorageBlobsDAOBuilderTest.java | 3 +- ...ftTempAuthObjectStorageBlobsDAOBuilderTest.java | 3 +- .../ObjectStorageDependenciesModule.java | 2 - .../swift/ObjectStorageBlobStoreModuleTest.java| 4 - 13 files changed, 211 insertions(+), 79 deletions(-) create mode 100644 server/blob/blob-objectstorage/src/test/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAOAWSTest.java - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Created] (JAMES-2823) Some cassandra tests don't work when launched individually
Matthieu Baechler created JAMES-2823: Summary: Some cassandra tests don't work when launched individually Key: JAMES-2823 URL: https://issues.apache.org/jira/browse/JAMES-2823 Project: James Server Issue Type: Bug Components: cassandra Reporter: Matthieu Baechler Some tests don't work as expected when run outside the CI. They fail when trying to access CassandraSchemaVersion table but it's not initialized by the Cassandra extension. On CI, we reuse the same tables but just empty them so the tests pass but it adds a dependency on execution order, which is to avoid. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Created] (JAMES-2822) Document of administratorId in usersrepository.xml
Gautier DI FOLCO created JAMES-2822: --- Summary: Document of administratorId in usersrepository.xml Key: JAMES-2822 URL: https://issues.apache.org/jira/browse/JAMES-2822 Project: James Server Issue Type: Improvement Reporter: Gautier DI FOLCO usersrepository.xml documentation is incomplete, fill the lacks of administratorId -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Created] (JAMES-2821) No logs on in memory product
Rémi Kowalski created JAMES-2821: Summary: No logs on in memory product Key: JAMES-2821 URL: https://issues.apache.org/jira/browse/JAMES-2821 Project: James Server Issue Type: Bug Reporter: Rémi Kowalski Fix For: 3.4.0 Logback dependency is missing -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Created] (JAMES-2820) Reduce defer in UnionBlobStore
Gautier DI FOLCO created JAMES-2820: --- Summary: Reduce defer in UnionBlobStore Key: JAMES-2820 URL: https://issues.apache.org/jira/browse/JAMES-2820 Project: James Server Issue Type: Improvement Reporter: Gautier DI FOLCO We use *Mono.defer* too much in *UnionBlobStore*. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[jira] [Created] (JAMES-2819) Upgrade pdfbox following CVE-2019-0228
Matthieu Baechler created JAMES-2819: Summary: Upgrade pdfbox following CVE-2019-0228 Key: JAMES-2819 URL: https://issues.apache.org/jira/browse/JAMES-2819 Project: James Server Issue Type: Task Affects Versions: 3.3.0, 3.4.0 Reporter: Matthieu Baechler James uses pdfbox library and a vulnerability was disclosed recently : CVE-2019-0228 We have to update pdfbox version in supported branches -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] branch master updated (8e3eb60 -> 92da31b)
This is an automated email from the ASF dual-hosted git repository. matthieu pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git. from 8e3eb60 JAMES-2272 enforce responsibilities segregation for Worker/Manager/WorkQueue new 514cc90 JAMES-2815 Stop storing twice in AWS S3 BlobStore new 92da31b JAMES-2815 Store directly small blobs in ObjectStore The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../{PutBlobFunction.java => BlobPutter.java} | 11 +- .../blob/objectstorage/ObjectStorageBlobsDAO.java | 55 --- .../ObjectStorageBlobsDAOBuilder.java | 14 +- .../objectstorage/StreamCompatibleBlobPutter.java | 55 +++ .../blob/objectstorage/aws/AwsS3ObjectStorage.java | 169 - .../aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java | 2 +- .../ObjectStorageDependenciesModule.java | 6 +- 7 files changed, 205 insertions(+), 107 deletions(-) rename server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/{PutBlobFunction.java => BlobPutter.java} (81%) create mode 100644 server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/StreamCompatibleBlobPutter.java - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org
[james-project] 01/02: JAMES-2815 Stop storing twice in AWS S3 BlobStore
This is an automated email from the ASF dual-hosted git repository. matthieu pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 514cc902da64e982d105fba844edc2597287cd13 Author: Gautier DI FOLCO AuthorDate: Fri Jul 5 11:34:22 2019 +0200 JAMES-2815 Stop storing twice in AWS S3 BlobStore --- .../{PutBlobFunction.java => BlobPutter.java} | 11 +- .../blob/objectstorage/ObjectStorageBlobsDAO.java | 33 +--- .../ObjectStorageBlobsDAOBuilder.java | 14 +- .../objectstorage/StreamCompatibleBlobPutter.java | 55 +++ .../blob/objectstorage/aws/AwsS3ObjectStorage.java | 169 - .../aws/AwsS3ObjectStorageBlobsDAOBuilderTest.java | 2 +- .../ObjectStorageDependenciesModule.java | 6 +- 7 files changed, 181 insertions(+), 109 deletions(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/PutBlobFunction.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/BlobPutter.java similarity index 81% rename from server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/PutBlobFunction.java rename to server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/BlobPutter.java index 6bae45d..a48c8f2 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/PutBlobFunction.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/BlobPutter.java @@ -19,6 +19,9 @@ package org.apache.james.blob.objectstorage; +import java.util.function.Supplier; + +import org.apache.james.blob.api.BlobId; import org.apache.james.blob.api.BucketName; import org.jclouds.blobstore.domain.Blob; @@ -31,8 +34,10 @@ import org.jclouds.blobstore.domain.Blob; * whereas you don't need one by using the S3 client. * */ -@FunctionalInterface -public interface PutBlobFunction { -void putBlob(BucketName bucketName, Blob blob); +public interface BlobPutter { + +void putDirectly(BucketName bucketName, Blob blob); + +BlobId putAndComputeId(BucketName bucketName, Blob initialBlob, Supplier blobIdSupplier); } diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java index cdfc166..394529d 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java @@ -22,6 +22,7 @@ package org.apache.james.blob.objectstorage; import java.io.IOException; import java.io.InputStream; import java.util.Optional; +import java.util.function.Supplier; import org.apache.commons.io.IOUtils; import org.apache.james.blob.api.BlobId; @@ -34,7 +35,6 @@ import org.apache.james.blob.objectstorage.swift.SwiftKeystone2ObjectStorage; import org.apache.james.blob.objectstorage.swift.SwiftKeystone3ObjectStorage; import org.apache.james.blob.objectstorage.swift.SwiftTempAuthObjectStorage; import org.jclouds.blobstore.domain.Blob; -import org.jclouds.blobstore.options.CopyOptions; import org.jclouds.domain.Location; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -55,17 +55,17 @@ public class ObjectStorageBlobsDAO implements BlobStore { private final BucketName defaultBucketName; private final org.jclouds.blobstore.BlobStore blobStore; -private final PutBlobFunction putBlobFunction; +private final BlobPutter blobPutter; private final PayloadCodec payloadCodec; ObjectStorageBlobsDAO(BucketName defaultBucketName, BlobId.Factory blobIdFactory, org.jclouds.blobstore.BlobStore blobStore, - PutBlobFunction putBlobFunction, + BlobPutter blobPutter, PayloadCodec payloadCodec) { this.blobIdFactory = blobIdFactory; this.defaultBucketName = defaultBucketName; this.blobStore = blobStore; -this.putBlobFunction = putBlobFunction; +this.blobPutter = blobPutter; this.payloadCodec = payloadCodec; } @@ -103,7 +103,7 @@ public class ObjectStorageBlobsDAO implements BlobStore { .contentLength(payload.getLength().orElse(new Long(data.length))) .build(); -return save(bucketName, blob) +return Mono.fromRunnable(() -> blobPutter.putDirectly(bucketName, blob)) .thenReturn(blobId); } @@ -112,31 +112,14 @@ public class ObjectStorageBlobsDAO implements BlobStore { Preconditions.checkNotNull(data); BlobId tmpId = blobIdFactory.randomId(); -return save(bucketName, data, tmpId) -.flatMap(id -> updateBlobId(bucketName,
[james-project] 02/02: JAMES-2815 Store directly small blobs in ObjectStore
This is an automated email from the ASF dual-hosted git repository. matthieu pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/james-project.git commit 92da31b306bdc92e483e6b95043bfc02e7c331cd Author: Gautier DI FOLCO AuthorDate: Fri Jul 5 11:51:08 2019 +0200 JAMES-2815 Store directly small blobs in ObjectStore --- .../blob/objectstorage/ObjectStorageBlobsDAO.java | 28 +- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java index 394529d..94f63fe 100644 --- a/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java +++ b/server/blob/blob-objectstorage/src/main/java/org/apache/james/blob/objectstorage/ObjectStorageBlobsDAO.java @@ -19,6 +19,7 @@ package org.apache.james.blob.objectstorage; +import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; import java.util.Optional; @@ -49,7 +50,7 @@ import reactor.core.scheduler.Schedulers; public class ObjectStorageBlobsDAO implements BlobStore { private static final Location DEFAULT_LOCATION = null; private static final Logger LOGGER = LoggerFactory.getLogger(ObjectStorageBlobsDAO.class); - +private static final int BUFFERED_SIZE = 256 * 1024; private final BlobId.Factory blobIdFactory; @@ -111,6 +112,31 @@ public class ObjectStorageBlobsDAO implements BlobStore { public Mono save(BucketName bucketName, InputStream data) { Preconditions.checkNotNull(data); +return Mono.defer(() -> savingStrategySelection(bucketName, data)); +} + +private Mono savingStrategySelection(BucketName bucketName, InputStream data) { +InputStream bufferedData = new BufferedInputStream(data, BUFFERED_SIZE + 1); +try { +if (isItABigStream(bufferedData)) { +return saveBigStream(bucketName, bufferedData); +} else { +return save(bucketName, IOUtils.toByteArray(bufferedData)); +} +} catch (IOException e) { +throw new RuntimeException(e); +} +} + +private boolean isItABigStream(InputStream bufferedData) throws IOException { +bufferedData.mark(0); +bufferedData.skip(BUFFERED_SIZE); +boolean isItABigStream = bufferedData.read() != -1; +bufferedData.reset(); +return isItABigStream; +} + +private Mono saveBigStream(BucketName bucketName, InputStream data) { BlobId tmpId = blobIdFactory.randomId(); HashingInputStream hashingInputStream = new HashingInputStream(Hashing.sha256(), data); Payload payload = payloadCodec.write(hashingInputStream); - To unsubscribe, e-mail: server-dev-unsubscr...@james.apache.org For additional commands, e-mail: server-dev-h...@james.apache.org