diff --git a/azure-storage/src/main/resources/reference.conf b/azure-storage/src/main/resources/reference.conf new file mode 100644 index 0000000000..75aab3ad98 --- /dev/null +++ b/azure-storage/src/main/resources/reference.conf @@ -0,0 +1,46 @@ +alpakka { + azure-storage { + api-version = "2024-11-04" + api-version = ${?AZURE_STORAGE_API_VERSION} + signing-algorithm = "HmacSHA256" + + # for local testing via emulator + # endpoint-url = "" + + #azure-credentials + credentials { + # valid values are anon (annonymous), SharedKey, and sas + authorization-type = anon + authorization-type = ${?AZURE_STORAGE_AUTHORIZATION_TYPE} + + # required for all authorization types + account-name = "" + account-name = ${?AZURE_STORAGE_ACCOUNT_NAME} + + # Account key is required for SharedKey or SharedKeyLite authorization + account-key = none + account-key = ${?AZURE_STORAGE_ACCOUNT_KEY} + + # SAS token for sas authorization + sas-token = "" + sas-token = ${?AZURE_STORAGE_SAS_TOKEN} + } + #azure-credentials + + # Default settings corresponding to automatic retry of requests in an Azure Blob Storage stream. + retry-settings { + # The maximum number of additional attempts (following transient errors) that will be made to process a given + # request before giving up. + max-retries = 3 + + # The minimum delay between request retries. + min-backoff = 200ms + + # The maximum delay between request retries. + max-backoff = 10s + + # Random jitter factor applied to retry delay calculation. + random-factor = 0.0 + } + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageAttributes.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageAttributes.scala new file mode 100644 index 0000000000..7d1918b51c --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageAttributes.scala @@ -0,0 +1,45 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.stream.Attributes +import akka.stream.Attributes.Attribute + +/** + * Akka Stream attributes that are used when materializing AzureStorage stream blueprints. + */ +object StorageAttributes { + + /** + * Settings to use for the Azure Blob Storage stream + */ + def settings(settings: StorageSettings): Attributes = Attributes(StorageSettingsValue(settings)) + + /** + * Config path which will be used to resolve required AzureStorage settings + */ + def settingsPath(path: String): Attributes = Attributes(StorageSettingsPath(path)) + + /** + * Default settings + */ + def defaultSettings: Attributes = Attributes(StorageSettingsPath.Default) +} + +final class StorageSettingsPath private (val path: String) extends Attribute + +object StorageSettingsPath { + val Default: StorageSettingsPath = StorageSettingsPath(StorageSettings.ConfigPath) + + def apply(path: String) = new StorageSettingsPath(path) +} + +final class StorageSettingsValue private (val settings: StorageSettings) extends Attribute + +object StorageSettingsValue { + def apply(settings: StorageSettings) = new StorageSettingsValue(settings) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageException.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageException.scala new file mode 100644 index 0000000000..40a930d85c --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageException.scala @@ -0,0 +1,80 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.http.scaladsl.model.StatusCode + +import scala.util.{Failure, Success, Try} +import scala.xml.{Elem, NodeSeq, XML} + +final case class StorageException(statusCode: StatusCode, + errorCode: String, + errorMessage: String, + resourceName: Option[String], + resourceValue: Option[String], + reason: Option[String]) + extends RuntimeException(errorMessage) { + + override def toString: String = + s"""StorageException( + |statusCode=$statusCode, + | errorCode=$errorCode, + | errorMessage=$errorMessage, + | resourceName=$resourceName, + | resourceValue=$resourceValue, + | reason=$reason + |)""".stripMargin.replaceAll(System.lineSeparator(), "") +} + +object StorageException { + def apply(statusCode: StatusCode, + errorCode: String, + errorMessage: String, + resourceName: Option[String], + resourceValue: Option[String], + reason: Option[String]): StorageException = + new StorageException(statusCode, errorCode, errorMessage, resourceName, resourceValue, reason) + + def apply(response: String, statusCode: StatusCode): StorageException = { + def getOptionalValue(xmlResponse: Elem, elementName: String, fallBackElementName: Option[String]) = { + val element = xmlResponse \ elementName + val node = + if (element.nonEmpty) element + else if (fallBackElementName.nonEmpty) xmlResponse \ fallBackElementName.get + else NodeSeq.Empty + + emptyStringToOption(node.text) + } + + Try { + val utf8_bom = "\uFEFF" + val sanitizedResponse = if (response.startsWith(utf8_bom)) response.substring(1) else response + val xmlResponse = XML.loadString(sanitizedResponse) + StorageException( + statusCode = statusCode, + errorCode = (xmlResponse \ "Code").text, + errorMessage = (xmlResponse \ "Message").text, + resourceName = getOptionalValue(xmlResponse, "QueryParameterName", Some("HeaderName")), + resourceValue = getOptionalValue(xmlResponse, "QueryParameterValue", Some("HeaderValue")), + reason = getOptionalValue(xmlResponse, "Reason", Some("AuthenticationErrorDetail")) + ) + } match { + case Failure(ex) => + val errorMessage = emptyStringToOption(ex.getMessage) + StorageException( + statusCode = statusCode, + errorCode = errorMessage.getOrElse("null"), + errorMessage = emptyStringToOption(response).orElse(errorMessage).getOrElse("null"), + resourceName = None, + resourceValue = None, + reason = None + ) + case Success(value) => value + } + + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageExt.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageExt.scala new file mode 100644 index 0000000000..7730aa964c --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/StorageExt.scala @@ -0,0 +1,42 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.actor.{ + ActorSystem, + ClassicActorSystemProvider, + ExtendedActorSystem, + Extension, + ExtensionId, + ExtensionIdProvider +} + +/** + * Manages one [[StorageSettings]] per `ActorSystem`. + */ +final class StorageExt private (sys: ExtendedActorSystem) extends Extension { + val settings: StorageSettings = settings(StorageSettings.ConfigPath) + + def settings(prefix: String): StorageSettings = StorageSettings(sys.settings.config.getConfig(prefix)) +} + +object StorageExt extends ExtensionId[StorageExt] with ExtensionIdProvider { + override def lookup: StorageExt.type = StorageExt + override def createExtension(system: ExtendedActorSystem) = new StorageExt(system) + + /** + * Java API. + * Get the Storage extension with the classic actors API. + */ + override def get(system: ActorSystem): StorageExt = super.apply(system) + + /** + * Java API. + * Get the Storage extension with the new actors API. + */ + override def get(system: ClassicActorSystemProvider): StorageExt = super.apply(system) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/ServerSideEncryption.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/ServerSideEncryption.scala new file mode 100644 index 0000000000..e1e092e18d --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/ServerSideEncryption.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package headers + +import akka.annotation.InternalApi +import akka.http.scaladsl.model.HttpHeader +import akka.http.scaladsl.model.headers.RawHeader + +import java.security.MessageDigest +import java.util.{Base64, Objects} + +sealed abstract class ServerSideEncryption { + @InternalApi private[storage] def headers: Seq[HttpHeader] +} + +object ServerSideEncryption { + def customerKey(key: String, hash: Option[String]): ServerSideEncryption = new CustomerKey(key, hash) + def customerKey(key: String): ServerSideEncryption = customerKey(key, None) +} + +final class CustomerKey private[headers] (val key: String, val hash: Option[String] = None) + extends ServerSideEncryption { + override private[storage] def headers: Seq[HttpHeader] = Seq( + RawHeader("x-ms-encryption-algorithm", "AES256"), + RawHeader("x-ms-encryption-key", key), + RawHeader("x-ms-encryption-key-sha256", hash.getOrElse(createHash)) + ) + + override def equals(obj: Any): Boolean = + obj match { + case other: CustomerKey => key == other.key && hash == other.hash + case _ => false + } + + override def hashCode(): Int = Objects.hash(key, hash) + + override def toString: String = + s"""ServerSideEncryption.CustomerKeys( + |key=$key, + | hash=$hash + |) + |""".stripMargin.replaceAll(System.lineSeparator(), "") + + private def createHash = { + val messageDigest = MessageDigest.getInstance("SHA-256") + val decodedKey = messageDigest.digest(Base64.getDecoder.decode(key)) + Base64.getEncoder.encodeToString(decodedKey) + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/headers.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/headers.scala new file mode 100644 index 0000000000..23a0f9fbef --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/headers/headers.scala @@ -0,0 +1,53 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package headers + +import akka.annotation.InternalApi +import akka.http.scaladsl.model.{ContentType, HttpHeader} +import akka.http.scaladsl.model.headers.{CustomHeader, RawHeader} + +private[storage] case class CustomContentTypeHeader(contentType: ContentType) extends CustomHeader { + override def name(): String = "Content-Type" + + override def value(): String = contentType.value + + override def renderInRequests(): Boolean = true + + override def renderInResponses(): Boolean = true +} + +private[storage] case class CustomContentLengthHeader(contentLength: Long) extends CustomHeader { + override def name(): String = "Content-Length" + + override def value(): String = contentLength.toString + + override def renderInRequests(): Boolean = true + + override def renderInResponses(): Boolean = true +} + +private[storage] case class BlobTypeHeader(blobType: String) { + @InternalApi private[storage] def header: HttpHeader = RawHeader(BlobTypeHeaderKey, blobType) +} + +object BlobTypeHeader { + private[storage] val BlockBlobHeader = new BlobTypeHeader(BlockBlobType) + private[storage] val PageBlobHeader = new BlobTypeHeader(PageBlobType) + private[storage] val AppendBlobHeader = new BlobTypeHeader(AppendBlobType) +} + +private[storage] case class RangeWriteTypeHeader(headerName: String, writeType: String) { + @InternalApi private[storage] def header: HttpHeader = RawHeader(headerName, writeType) +} + +object RangeWriteTypeHeader { + private[storage] val UpdateFileHeader = new RangeWriteTypeHeader(FileWriteTypeHeaderKey, "update") + private[storage] val ClearFileHeader = new RangeWriteTypeHeader(FileWriteTypeHeaderKey, "clear") + private[storage] val UpdatePageHeader = new RangeWriteTypeHeader(PageWriteTypeHeaderKey, "update") + private[storage] val ClearPageHeader = new RangeWriteTypeHeader(PageWriteTypeHeaderKey, "clear") +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/AzureStorageStream.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/AzureStorageStream.scala new file mode 100644 index 0000000000..4e2dab42c2 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/AzureStorageStream.scala @@ -0,0 +1,282 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package impl + +import akka.NotUsed +import akka.actor.ActorSystem +import akka.http.scaladsl.Http +import akka.http.scaladsl.model.StatusCodes.{Accepted, Created, NotFound, OK} +import akka.http.scaladsl.model.headers.{`Content-Length`, `Content-Type`} +import akka.http.scaladsl.model.{ + HttpEntity, + HttpHeader, + HttpRequest, + HttpResponse, + MessageEntity, + ResponseEntity, + StatusCode +} +import akka.http.scaladsl.unmarshalling.Unmarshal +import akka.stream.alpakka.azure.storage.headers.CustomContentTypeHeader +import akka.stream.{Attributes, Materializer} +import akka.stream.alpakka.azure.storage.impl.auth.Signer +import akka.stream.alpakka.azure.storage.requests.{ + ClearFileRange, + CreateContainer, + CreateDirectory, + CreateFile, + DeleteContainer, + DeleteDirectory, + GetProperties, + RequestBuilder, + UpdateFileRange +} +import akka.stream.scaladsl.{Flow, RetryFlow, Source} +import akka.util.ByteString + +import java.time.Clock +import scala.concurrent.{ExecutionContext, Future, Promise} +import scala.util.{Failure, Success, Try} + +object AzureStorageStream { + + private[storage] def getObject(storageType: String, + objectPath: String, + requestBuilder: RequestBuilder): Source[ByteString, Future[ObjectMetadata]] = + Source + .fromMaterializer { (mat, attr) => + implicit val system: ActorSystem = mat.system + val settings = resolveSettings(attr, system) + val request = requestBuilder.createRequest(settings, storageType, objectPath) + val objectMetadataMat = Promise[ObjectMetadata]() + signAndRequest(request, settings)(mat.system) + .map(response => response.withEntity(response.entity.withoutSizeLimit)) + .mapAsync(parallelism = 1)(entityForSuccess) + .flatMapConcat { + case (entity, headers) => + objectMetadataMat.success(computeMetaData(headers, entity)) + entity.dataBytes + } + .mapError { + case e: Throwable => + objectMetadataMat.tryFailure(e) + e + } + .mapMaterializedValue(_ => objectMetadataMat.future) + } + .mapMaterializedValue(_.flatMap(identity)(ExecutionContext.parasitic)) + + private[storage] def getBlobProperties(objectPath: String, + requestBuilder: GetProperties): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = OK, storageType = BlobType, objectPath = objectPath, requestBuilder = requestBuilder) + + private[storage] def getFileProperties(objectPath: String, + requestBuilder: GetProperties): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = OK, storageType = FileType, objectPath = objectPath, requestBuilder = requestBuilder) + + private[storage] def deleteBlob(objectPath: String, + requestBuilder: RequestBuilder): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Accepted, + storageType = BlobType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def deleteFile(objectPath: String, + requestBuilder: RequestBuilder): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Accepted, + storageType = FileType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def putBlob(objectPath: String, + requestBuilder: RequestBuilder, + maybeHttpEntity: Option[MessageEntity]): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Created, + storageType = BlobType, + objectPath = objectPath, + requestBuilder = requestBuilder, + maybeHttpEntity = maybeHttpEntity) + + private[storage] def createFile(objectPath: String, + requestBuilder: CreateFile): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Created, + storageType = FileType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def updateRange(objectPath: String, + httpEntity: MessageEntity, + requestBuilder: UpdateFileRange): Source[Option[ObjectMetadata], NotUsed] = + handleRequest( + successCode = Created, + storageType = FileType, + objectPath = objectPath, + requestBuilder = requestBuilder, + maybeHttpEntity = Some(httpEntity) + ) + + private[storage] def clearRange(objectPath: String, + requestBuilder: ClearFileRange): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Created, + storageType = FileType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def createContainer(objectPath: String, + requestBuilder: CreateContainer): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Created, + storageType = BlobType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def deleteContainer(objectPath: String, + requestBuilder: DeleteContainer): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Accepted, + storageType = BlobType, + objectPath = objectPath, + requestBuilder = requestBuilder) + + private[storage] def createDirectory(directoryPath: String, + requestBuilder: CreateDirectory): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Created, + storageType = FileType, + objectPath = directoryPath, + requestBuilder = requestBuilder) + + private[storage] def deleteDirectory(directoryPath: String, + requestBuilder: DeleteDirectory): Source[Option[ObjectMetadata], NotUsed] = + handleRequest(successCode = Accepted, + storageType = FileType, + objectPath = directoryPath, + requestBuilder = requestBuilder) + + /** + * Common function to handle all requests where we don't expect response body. + * + * @param successCode status code for successful response + * @param storageType storage type + * @param objectPath path of the object. + * @param requestBuilder request builder + * @param maybeHttpEntity optional http entity + * @return Source with metadata containing response headers + */ + private def handleRequest( + successCode: StatusCode, + storageType: String, + objectPath: String, + requestBuilder: RequestBuilder, + maybeHttpEntity: Option[MessageEntity] = None + ): Source[Option[ObjectMetadata], NotUsed] = + Source + .fromMaterializer { (mat, attr) => + implicit val system: ActorSystem = mat.system + import system.dispatcher + val settings = resolveSettings(attr, system) + val httpEntity = maybeHttpEntity.getOrElse(HttpEntity.Empty) + val request = requestBuilder.createRequest(settings, storageType, objectPath).withEntity(httpEntity) + signAndRequest(request, settings).flatMapConcat { + case HttpResponse(sc, h, entity, _) if sc == successCode => + Source.future(entity.withoutSizeLimit().discardBytes().future().map(_ => Some(computeMetaData(h, entity)))) + case HttpResponse(NotFound, _, entity, _) => + Source.future( + entity + .withoutSizeLimit() + .discardBytes() + .future() + .map(_ => None)(ExecutionContext.parasitic) + ) + case response: HttpResponse => Source.future(unmarshalError(response.status, response.entity)) + } + } + .mapMaterializedValue(_ => NotUsed) + + private def signAndRequest( + request: HttpRequest, + settings: StorageSettings + )(implicit + system: ActorSystem): Source[HttpResponse, NotUsed] = { + import system.dispatcher + + val retriableFlow = Flow[HttpRequest] + .mapAsync(parallelism = 1)( + req => + Http() + .singleRequest(req) + .map(Success.apply) + .recover[Try[HttpResponse]] { + case t => + Failure(t) + } + ) + + import settings.retrySettings._ + implicit val clock: Clock = Clock.systemUTC() + Source + .single(request) + .flatMapConcat(request => Signer(request, settings).signedRequest) + .via(RetryFlow.withBackoff(minBackoff, maxBackoff, randomFactor, maxRetries, retriableFlow) { + case (request, Success(response)) if isTransientError(response.status) => + response.entity.discardBytes() + Some(request) + case (request, Failure(_)) => + // Treat any exception as transient. + Some(request) + case _ => None + }) + .mapAsync(1)(Future.fromTry) + } + + private def computeMetaData(headers: Seq[HttpHeader], entity: ResponseEntity): ObjectMetadata = { + val contentLengthHeader = `Content-Length` + .parseFromValueString(entity.contentLengthOption.getOrElse(0L).toString) + .map(Seq(_)) + .getOrElse(Nil) + val contentTypeHeader = `Content-Type` + .parseFromValueString(entity.contentType.value) + .map(Seq(_)) + .getOrElse(Nil) + ObjectMetadata( + headers ++ contentLengthHeader ++ contentTypeHeader ++ Seq(CustomContentTypeHeader(entity.contentType)) + ) + } + + private def isTransientError(status: StatusCode): Boolean = { + // 5xx errors from Azure Blob Storage can be treated as transient. + status.intValue >= 500 + } + + private def unmarshalError(code: StatusCode, entity: ResponseEntity)(implicit mat: Materializer) = { + import mat.executionContext + Unmarshal(entity).to[String].map { err => + throw StorageException(err, code) + } + } + + private def entityForSuccess( + resp: HttpResponse + )(implicit mat: Materializer): Future[(ResponseEntity, Seq[HttpHeader])] = { + resp match { + case HttpResponse(status, headers, entity, _) if status.isSuccess() && !status.isRedirection() => + Future.successful((entity, headers)) + case response: HttpResponse => + unmarshalError(response.status, response.entity) + } + } + + private def resolveSettings(attr: Attributes, sys: ActorSystem) = + attr + .get[StorageSettingsValue] + .map(_.settings) + .getOrElse { + val storageExtension = StorageExt(sys) + attr + .get[StorageSettingsPath] + .map(settingsPath => storageExtension.settings(settingsPath.path)) + .getOrElse(storageExtension.settings) + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/StorageHeaders.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/StorageHeaders.scala new file mode 100644 index 0000000000..5b48a5e0b4 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/StorageHeaders.scala @@ -0,0 +1,172 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package impl + +import akka.http.scaladsl.model.headers.{ByteRange, RawHeader, Range => RangeHeader} +import akka.http.scaladsl.model.{ContentType, HttpHeader} +import akka.stream.alpakka.azure.storage.headers._ + +import java.util.Objects + +private[storage] class StorageHeaders private (val contentLengthHeader: Option[HttpHeader] = None, + val contentTypeHeader: Option[HttpHeader] = None, + val sse: Option[ServerSideEncryption] = None, + val rangeHeader: Option[HttpHeader] = None, + val blobTypeHeader: Option[HttpHeader] = None, + val leaseIdHeader: Option[HttpHeader] = None, + val fileWriteTypeHeader: Option[HttpHeader] = None, + val rangeWriteTypeHeader: Option[HttpHeader] = None, + val fileMaxContentLengthHeader: Option[HttpHeader] = None, + val pageBlobContentLengthHeader: Option[HttpHeader] = None, + val pageBlobSequenceNumberHeader: Option[HttpHeader] = None, + val additionalHeaders: Seq[HttpHeader] = Seq.empty) { + + private[storage] def headers: Seq[HttpHeader] = + (contentLengthHeader ++ + contentTypeHeader ++ + rangeHeader ++ + blobTypeHeader ++ + leaseIdHeader ++ + fileWriteTypeHeader ++ + rangeWriteTypeHeader ++ + fileMaxContentLengthHeader ++ + pageBlobContentLengthHeader ++ + pageBlobSequenceNumberHeader).toSeq ++ sse.map(_.headers).getOrElse(Seq.empty) ++ additionalHeaders + + private[storage] def withContentLengthHeader(contentLength: Long): StorageHeaders = + copy(contentLengthHeader = Some(CustomContentLengthHeader(contentLength))) + + private[storage] def withContentTypeHeader(contentType: ContentType): StorageHeaders = + copy(contentTypeHeader = Some(CustomContentTypeHeader(contentType))) + + private[storage] def witServerSideEncryption(sse: Option[ServerSideEncryption]): StorageHeaders = + copy(sse = sse) + + private[storage] def withRangeHeader(range: ByteRange): StorageHeaders = + copy(rangeHeader = Some(RangeHeader(range))) + + private[storage] def withRangeHeader(range: Option[ByteRange]): StorageHeaders = + copy(rangeHeader = range.map(value => RangeHeader(value))) + + private[storage] def withBlobTypeHeader(blobTypeHeader: BlobTypeHeader): StorageHeaders = + copy(blobTypeHeader = Some(blobTypeHeader.header)) + + private[storage] def withLeaseIdHeader(leaseId: Option[String]): StorageHeaders = + copy(leaseIdHeader = leaseId.map(value => RawHeader(LeaseIdHeaderKey, value))) + + private[storage] def withRangeWriteTypeHeader(fileWriteTypeHeader: RangeWriteTypeHeader): StorageHeaders = + copy(rangeWriteTypeHeader = Some(fileWriteTypeHeader.header)) + + private[storage] def withFileTypeHeader(): StorageHeaders = + copy(fileTypeHeader = Some(RawHeader(FileTypeHeaderKey, "file"))) + + private[storage] def withFileMaxContentLengthHeader(contentLength: Long): StorageHeaders = + copy(fileMaxContentLengthHeader = Some(RawHeader(XMsContentLengthHeaderKey, contentLength.toString))) + + private[storage] def withPageBlobContentLengthHeader(contentLength: Long): StorageHeaders = + copy(pageBlobContentLengthHeader = Some(RawHeader(PageBlobContentLengthHeaderKey, contentLength.toString))) + + private[storage] def withPageBlobSequenceNumberHeader(sequenceNumber: Option[Int]): StorageHeaders = + copy( + pageBlobSequenceNumberHeader = + sequenceNumber.map(value => RawHeader(PageBlobSequenceNumberHeaderKey, value.toString)) + ) + + private[storage] def withAdditionalHeaders(additionalHeaders: Seq[HttpHeader]) = + copy(additionalHeaders = additionalHeaders) + + private[storage] def addHeader(httpHeader: HttpHeader): StorageHeaders = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(contentLengthHeader: Option[HttpHeader] = contentLengthHeader, + contentTypeHeader: Option[HttpHeader] = contentTypeHeader, + rangeHeader: Option[HttpHeader] = rangeHeader, + sse: Option[ServerSideEncryption] = sse, + blobTypeHeader: Option[HttpHeader] = blobTypeHeader, + leaseIdHeader: Option[HttpHeader] = leaseIdHeader, + rangeWriteTypeHeader: Option[HttpHeader] = fileWriteTypeHeader, + fileTypeHeader: Option[HttpHeader] = rangeWriteTypeHeader, + fileMaxContentLengthHeader: Option[HttpHeader] = fileMaxContentLengthHeader, + pageBlobContentLengthHeader: Option[HttpHeader] = pageBlobContentLengthHeader, + pageBlobSequenceNumberHeader: Option[HttpHeader] = pageBlobSequenceNumberHeader, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new StorageHeaders( + contentLengthHeader = contentLengthHeader, + contentTypeHeader = contentTypeHeader, + sse = sse, + rangeHeader = rangeHeader, + blobTypeHeader = blobTypeHeader, + leaseIdHeader = leaseIdHeader, + fileWriteTypeHeader = rangeWriteTypeHeader, + rangeWriteTypeHeader = fileTypeHeader, + fileMaxContentLengthHeader = fileMaxContentLengthHeader, + pageBlobContentLengthHeader = pageBlobContentLengthHeader, + pageBlobSequenceNumberHeader = pageBlobSequenceNumberHeader, + additionalHeaders = additionalHeaders + ) + + override def toString: String = + s"""StorageHeaders( + |contentLengthHeader=${contentLengthHeader.map(_.value()).getOrElse("None")}, + | contentTypeHeader=${contentTypeHeader.map(_.value()).getOrElse("None")}, + | rangeHeader=${rangeHeader.map(_.value()).getOrElse("None")}, + | sse=${sse.toString}, + | blobTypeHeader=${blobTypeHeader.map(_.value()).getOrElse("None")}, + | leaseIdHeader=${leaseIdHeader.map(_.value()).getOrElse("None")}, + | fileWriteTypeHeader=${fileWriteTypeHeader.map(_.value()).getOrElse("None")}, + | fileTypeHeader=${rangeWriteTypeHeader.map(_.value()).getOrElse("None")}, + | fileMaxContentLengthHeader=${fileMaxContentLengthHeader.map(_.value()).getOrElse("None")}, + | pageBlobContentLengthHeader=${pageBlobContentLengthHeader.map(_.value()).getOrElse("None")}, + | pageBlobSequenceNumberHeader=${pageBlobSequenceNumberHeader.map(_.value()).getOrElse("None")}, + | additionalHeaders=${additionalHeaders.mkString("[", " ,", "]")} + |)""".stripMargin.replaceAll(System.lineSeparator(), "") + + override def equals(obj: Any): Boolean = + obj match { + case other: StorageHeaders => + Objects.equals(contentLengthHeader, other.contentLengthHeader) && + Objects.equals(contentTypeHeader, other.contentTypeHeader) && + Objects.equals(sse, other.sse) && + Objects.equals(rangeHeader, other.rangeHeader) && + Objects.equals(blobTypeHeader, other.blobTypeHeader) && + Objects.equals(leaseIdHeader, other.leaseIdHeader) && + Objects.equals(fileWriteTypeHeader, other.fileWriteTypeHeader) && + Objects.equals(fileMaxContentLengthHeader, other.fileMaxContentLengthHeader) && + Objects.equals(pageBlobContentLengthHeader, other.pageBlobContentLengthHeader) && + Objects.equals(pageBlobSequenceNumberHeader, other.pageBlobSequenceNumberHeader) && + Objects.equals(additionalHeaders, other.additionalHeaders) + case _ => false + } + + override def hashCode(): Int = + Objects.hash( + contentLengthHeader, + contentTypeHeader, + sse, + rangeHeader, + blobTypeHeader, + leaseIdHeader, + fileWriteTypeHeader, + rangeWriteTypeHeader, + fileMaxContentLengthHeader, + pageBlobContentLengthHeader, + pageBlobSequenceNumberHeader, + additionalHeaders + ) +} + +private[storage] object StorageHeaders { + private[storage] val Empty = new StorageHeaders() + + private[storage] def apply(): StorageHeaders = Empty + + /** + * Java Api + */ + def create(): StorageHeaders = Empty +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/auth/Signer.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/auth/Signer.scala new file mode 100644 index 0000000000..a716aca505 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/impl/auth/Signer.scala @@ -0,0 +1,122 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package impl.auth + +import akka.NotUsed +import akka.http.scaladsl.model.{HttpRequest, Uri} +import akka.http.scaladsl.model.headers._ +import akka.stream.scaladsl.Source + +import java.time.Clock +import java.util.Base64 +import javax.crypto.Mac +import javax.crypto.spec.SecretKeySpec + +/** Takes initial request and add signed `Authorization` header and essential XMS headers. + * + * @param initialRequest + * - Initial HTTP request without authorization and XMS headers + * @param settings + * - Storage settings + */ +final case class Signer(initialRequest: HttpRequest, settings: StorageSettings)(implicit clock: Clock) { + + private val credential = settings.azureNameKeyCredential + private val requestWithHeaders = + initialRequest + .addHeader(RawHeader(XmsDateHeaderKey, getFormattedDate)) + .addHeader(RawHeader(XmsVersionHeaderKey, settings.apiVersion)) + + private lazy val mac = { + val mac = Mac.getInstance(settings.algorithm) + mac.init(new SecretKeySpec(credential.accountKey, settings.algorithm)) + mac + } + + def signedRequest: Source[HttpRequest, NotUsed] = { + val authorizationType = settings.authorizationType + if (authorizationType == AnonymousAuthorizationType || authorizationType == SasAuthorizationType) + Source.single(requestWithHeaders) + else + Source.single( + requestWithHeaders.addHeader( + RawHeader(AuthorizationHeaderKey, generateAuthorizationHeader) + ) + ) + } + + private def getHeaderOptionalValue(headerName: String) = + requestWithHeaders.headers.collectFirst { + case header if header.name() == headerName => header.value() + } + + private def getHeaderValue(headerName: String, defaultValue: String = "") = + getHeaderOptionalValue(headerName).getOrElse(defaultValue) + + private def getContentLengthValue = { + val contentLengthValue = getHeaderValue(`Content-Length`.name, "0") + if (contentLengthValue == "0") "" else contentLengthValue + } + + private def buildHeadersToSign(headerNames: Seq[String]) = { + def getValue(headerName: String) = { + if (headerName == `Content-Length`.name) getContentLengthValue + else getHeaderValue(headerName) + } + + Seq(requestWithHeaders.method.value.toUpperCase) ++ headerNames.map(getValue) ++ + getAdditionalXmsHeaders ++ getCanonicalizedResource + } + + private def getAdditionalXmsHeaders = + requestWithHeaders.headers.filter(header => header.name().startsWith("x-ms-")).sortBy(_.name()).map { header => + s"${header.name().toLowerCase}:${header.value()}" + } + + private def getCanonicalizedResource = { + val uri = requestWithHeaders.uri + val resourcePath = s"/${credential.accountName}${uri.path.toString()}" + + val queries = + uri.queryString() match { + case Some(queryString) => + Uri.Query(queryString).toMap.toSeq.sortBy(_._1).map { + case (key, value) => s"${key.toLowerCase}:$value" + } + + case None => Seq.empty[String] + } + + Seq(resourcePath) ++ queries + } + + private[auth] def generateAuthorizationHeader: String = { + import Signer._ + val authorizationType = settings.authorizationType + val headersToSign = buildHeadersToSign(SharedKeyHeaders) + val signature = Base64.getEncoder.encodeToString(mac.doFinal(headersToSign.mkString(NewLine).getBytes)) + s"$authorizationType ${credential.accountName}:$signature" + } +} + +object Signer { + private val SharedKeyHeaders = + Seq( + `Content-Encoding`.name, + "Content-Language", + `Content-Length`.name, + "Content-MD5", + `Content-Type`.name, + Date.name, + `If-Modified-Since`.name, + `If-Match`.name, + `If-None-Match`.name, + `If-Unmodified-Since`.name, + Range.name + ) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/BlobService.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/BlobService.scala new file mode 100644 index 0000000000..266c1a4612 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/BlobService.scala @@ -0,0 +1,148 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package javadsl + +import akka.NotUsed +import akka.http.scaladsl.model.HttpEntity +import akka.stream.alpakka.azure.storage.impl.AzureStorageStream +import akka.stream.alpakka.azure.storage.requests.{ + CreateContainer, + DeleteContainer, + DeleteFile, + GetBlob, + GetProperties, + PutAppendBlock, + PutBlockBlob, + PutPageBlock +} +import akka.stream.javadsl.Source +import akka.stream.scaladsl.SourceToCompletionStage +import akka.util.ByteString + +import java.util.Optional +import java.util.concurrent.CompletionStage + +/** + * Java API for BlobService operations. + */ +object BlobService { + + /** + * Gets blob representing `objectPath` with specified range (if applicable). + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build getBlob properties request + * @return A [[akka.stream.javadsl.Source]] containing the objects data as a [[akka.util.ByteString]] along with a + * materialized value containing the [[akka.stream.alpakka.azure.storage.ObjectMetadata]] + */ + def getBlob(objectPath: String, requestBuilder: GetBlob): Source[ByteString, CompletionStage[ObjectMetadata]] = + new Source( + AzureStorageStream + .getObject(BlobType, objectPath, requestBuilder) + .toCompletionStage() + ) + + /** + * Gets blob properties. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build getBlob properties request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def getProperties(objectPath: String, requestBuilder: GetProperties): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .getBlobProperties(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Deletes blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build deleteFile request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteBlob(objectPath: String, requestBuilder: DeleteFile): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .deleteBlob(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Put Block blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build putBlockBlob request + * @param payload actual payload, a [[akka.stream.javadsl.Source Source]] of [[akka.util.ByteString ByteString]] + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putBlockBlob(objectPath: String, + requestBuilder: PutBlockBlob, + payload: Source[ByteString, _]): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .putBlob( + objectPath, + requestBuilder, + Some(HttpEntity(requestBuilder.contentType, requestBuilder.contentLength, payload.asScala)) + ) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Put (Create) Page Blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build putPageBlob request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putPageBlock(objectPath: String, requestBuilder: PutPageBlock): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .putBlob(objectPath, requestBuilder, None) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Put (Create) Append Blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build putAppendBlob request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putAppendBlock(objectPath: String, requestBuilder: PutAppendBlock): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .putBlob(objectPath, requestBuilder, None) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Create container. + * + * @param objectPath name of the container + * @param requestBuilder builder to build createContainer request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createContainer(objectPath: String, requestBuilder: CreateContainer): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream.createContainer(objectPath, requestBuilder).map(opt => Optional.ofNullable(opt.orNull)).asJava + + /** + * Delete container. + * + * @param objectPath name of the container + * @param requestBuilder builder to build deleteContainer request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteContainer(objectPath: String, requestBuilder: DeleteContainer): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteContainer(objectPath, requestBuilder).map(opt => Optional.ofNullable(opt.orNull)).asJava +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/FileService.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/FileService.scala new file mode 100644 index 0000000000..83a3eaeead --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/javadsl/FileService.scala @@ -0,0 +1,155 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package javadsl + +import akka.NotUsed +import akka.http.scaladsl.model.HttpEntity +import akka.stream.alpakka.azure.storage.impl.AzureStorageStream +import akka.stream.alpakka.azure.storage.requests.{ + ClearFileRange, + CreateDirectory, + CreateFile, + DeleteDirectory, + DeleteFile, + GetFile, + GetProperties, + UpdateFileRange +} +import akka.stream.javadsl.Source +import akka.stream.scaladsl.SourceToCompletionStage +import akka.util.ByteString + +import java.util.Optional +import java.util.concurrent.CompletionStage + +/** + * Java API FileService operations + */ +object FileService { + + /** + * Gets file representing `objectPath` with specified range (if applicable). + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build getBlob request + * @return A [[akka.stream.javadsl.Source]] containing the objects data as a [[akka.util.ByteString]] along with a + * materialized value containing the [[akka.stream.alpakka.azure.storage.ObjectMetadata]] + */ + def getFile(objectPath: String, requestBuilder: GetFile): Source[ByteString, CompletionStage[ObjectMetadata]] = { + new Source( + AzureStorageStream + .getObject(FileType, objectPath, requestBuilder) + .toCompletionStage() + ) + } + + /** + * Gets file properties. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build getFile proroperties request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def getProperties(objectPath: String, requestBuilder: GetProperties): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .getFileProperties(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Deletes file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build getFile proroperties request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteFile(objectPath: String, requestBuilder: DeleteFile): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .deleteFile(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Creates a file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build createFile request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createFile(objectPath: String, requestBuilder: CreateFile): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .createFile(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Updates file on the specified range. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder range of bytes to be written + * @param payload actual payload, a [[akka.stream.javadsl.Source Source]] of [[akka.util.ByteString ByteString]] + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def updateRange(objectPath: String, + requestBuilder: UpdateFileRange, + payload: Source[ByteString, _]): Source[Optional[ObjectMetadata], NotUsed] = { + AzureStorageStream + .updateRange( + objectPath, + HttpEntity(requestBuilder.contentType, + requestBuilder.range.last - requestBuilder.range.first + 1, + payload.asScala), + requestBuilder + ) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + } + + /** + * Clears specified range from the file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build clearRange request + * @return A [[akka.stream.javadsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def clearRange(objectPath: String, requestBuilder: ClearFileRange): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream + .clearRange(objectPath, requestBuilder) + .map(opt => Optional.ofNullable(opt.orNull)) + .asJava + + /** + * Create directory. + * + * @param directoryPath path of the directory to be created, e.g., `myshare/myparentdirectorypath/mydirectory` + * @param requestBuilder builder to build createDirectory request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createDirectory(directoryPath: String, + requestBuilder: CreateDirectory): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream.createDirectory(directoryPath, requestBuilder).map(opt => Optional.ofNullable(opt.orNull)).asJava + + /** + * Delete directory. + * + * @param directoryPath path of the directory to be deleted, e.g., `myshare/myparentdirectorypath/mydirectory` + * @param requestBuilder builder to build deleteDirectory request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteDirectory(directoryPath: String, + requestBuilder: DeleteDirectory): Source[Optional[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteDirectory(directoryPath, requestBuilder).map(opt => Optional.ofNullable(opt.orNull)).asJava + +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/models.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/models.scala new file mode 100644 index 0000000000..c7e3de26f2 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/models.scala @@ -0,0 +1,187 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.http.scaladsl.model.{DateTime, HttpHeader} +import akka.http.scaladsl.model.headers._ +import com.typesafe.config.Config + +import java.util.{Base64, Optional} +import scala.jdk.CollectionConverters._ +import scala.jdk.OptionConverters._ + +final case class AzureNameKeyCredential(accountName: String, accountKey: Array[Byte]) + +object AzureNameKeyCredential { + def apply(accountName: String, accountKey: String): AzureNameKeyCredential = + AzureNameKeyCredential(accountName, Base64.getDecoder.decode(accountKey)) + + /** Java API */ + def create(accountName: String, accountKey: String): AzureNameKeyCredential = + AzureNameKeyCredential(accountName, accountKey) + + def apply(config: Config): AzureNameKeyCredential = { + val accountName = config.getString("account-name", "") + val accountKey = config.getString("account-key", "") + AzureNameKeyCredential(accountName, accountKey) + } +} + +/** Modelled after BlobProperties in Azure Blob Storage. + * + * @param metadata + * Raw Http headers + */ +final class ObjectMetadata private (val metadata: Seq[HttpHeader]) { + + /** Java Api + */ + lazy val headers: java.util.List[akka.http.javadsl.model.HttpHeader] = + (metadata: Seq[akka.http.javadsl.model.HttpHeader]).asJava + + /** + * Content MD5 + */ + lazy val contentMd5: Option[String] = metadata.collectFirst { + case e if e.name == "Content-MD5" => removeQuotes(e.value()) + } + + /** + * Java API + * Content MD5 + */ + lazy val getContentMd5: Optional[String] = contentMd5.toJava + + /** Gets the hex encoded 128-bit MD5 digest of the associated object according to RFC 1864. This data is used as an + * integrity check to verify that the data received by the caller is the same data that was sent by Azure Storage. + *

This field represents the hex encoded 128-bit MD5 digest of an object's content as calculated by Azure + * Storage. The ContentMD5 field represents the base64 encoded 128-bit MD5 digest as calculated on the caller's side. + *

+ * + * @return + * The hex encoded MD5 hash of the content for the associated object as calculated by Azure Storage. + */ + lazy val eTag: Option[String] = metadata.collectFirst { + case e: ETag => removeQuotes(e.etag.value) + } + + /** Java Api + * + * Gets the hex encoded 128-bit MD5 digest of the associated object according to RFC 1864. This data is used as an + * integrity check to verify that the data received by the caller is the same data that was sent by Azure Storage. + *

This field represents the hex encoded 128-bit MD5 digest of an object's content as calculated by Azure + * Storage. The ContentMD5 field represents the base64 encoded 128-bit MD5 digest as calculated on the caller's side. + *

+ * + * @return + * The hex encoded MD5 hash of the content for the associated object as calculated by Azure Storage. + */ + lazy val getETag: Optional[String] = eTag.toJava + + /**

Gets the Content-Length HTTP header indicating the size of the associated object in bytes.

This field + * is required when uploading objects to Storage, but the Azure Storage Java client will automatically set it when + * working directly with files. When uploading directly from a stream, set this field if possible. Otherwise the + * client must buffer the entire stream in order to calculate the content length before sending the data to Azure + * Storage.

For more information on the Content-Length HTTP header, see + * [[https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.13]]

+ * + * @return + * The Content-Length HTTP header indicating the size of the associated object in bytes. + * @see + * ObjectMetadata#setContentLength(long) + */ + lazy val contentLength: Long = + metadata + .collectFirst { + case cl: `Content-Length` => + cl.length + } + .getOrElse(0) + + /** Java Api + * + *

Gets the Content-Length HTTP header indicating the size of the associated object in bytes.

This field + * is required when uploading objects to Storage, but the Azure Storage Java client will automatically set it when + * working directly with files. When uploading directly from a stream, set this field if possible. Otherwise the + * client must buffer the entire stream in order to calculate the content length before sending the data to Azure + * Storage.

For more information on the Content-Length HTTP header, see + * [[https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.13]]

+ * + * @return + * The Content-Length HTTP header indicating the size of the associated object in bytes. + * @see + * ObjectMetadata#setContentLength(long) + */ + lazy val getContentLength: Long = contentLength + + /**

Gets the Content-Type HTTP header, which indicates the type of content stored in the associated object. The + * value of this header is a standard MIME type.

When uploading files, the Azure Storage Java client will + * attempt to determine the correct content type if one hasn't been set yet. Users are responsible for ensuring a + * suitable content type is set when uploading streams. If no content type is provided and cannot be determined by + * the filename, the default content type, "application/octet-stream", will be used.

For more information on + * the Content-Type header, see [[https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.17]]

+ * + * @return + * The HTTP Content-Type header, indicating the type of content stored in the associated Storage object. + * @see + * ObjectMetadata#setContentType(String) + */ + lazy val contentType: Option[String] = metadata.collectFirst { + case ct: `Content-Type` => + ct.value + } + + /** Java Api + * + *

Gets the Content-Type HTTP header, which indicates the type of content stored in the associated object. The + * value of this header is a standard MIME type.

When uploading files, the Azure Storage Java client will + * attempt to determine the correct content type if one hasn't been set yet. Users are responsible for ensuring a + * suitable content type is set when uploading streams. If no content type is provided and cannot be determined by + * the filename, the default content type, "application/octet-stream", will be used.

For more information on + * the Content-Type header, see [[https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.17]]

+ * + * @return + * The HTTP Content-Type header, indicating the type of content stored in the associated Storage object. + * @see + * ObjectMetadata#setContentType(String) + */ + lazy val getContentType: Optional[String] = contentType.toJava + + /** Gets the value of the Last-Modified header, indicating the date and time at which Azure Storage last recorded a + * modification to the associated object. + * + * @return + * The date and time at which Azure Storage last recorded a modification to the associated object. + */ + lazy val lastModified: Option[DateTime] = metadata.collectFirst { + case ct: `Last-Modified` => + ct.date + } + + /** Java Api + * + * Gets the value of the Last-Modified header, indicating the date and time at which Azure Storage last recorded a + * modification to the associated object. + * + * @return + * The date and time at which Azure Storage last recorded a modification to the associated object. + */ + lazy val getLastModified: Option[DateTime] = lastModified + + override def toString: String = + s"""ObjectMetadata( + |contentMd5=$contentMd5 + | eTag=$eTag, + | contentLength=$contentLength, + | contentType=$contentType, + | lastModified=$lastModified + |)""".stripMargin.replaceAll(System.lineSeparator(), "") +} + +object ObjectMetadata { + def apply(metadata: Seq[HttpHeader]) = new ObjectMetadata(metadata) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/package.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/package.scala new file mode 100644 index 0000000000..8cb17d77b0 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/package.scala @@ -0,0 +1,61 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure + +import com.typesafe.config.Config + +import java.time.{Clock, ZoneOffset} +import java.time.format.DateTimeFormatter + +package object storage { + + private[storage] val NewLine: String = "\n" + private[storage] val AuthorizationHeaderKey = "Authorization" + private[storage] val XmsDateHeaderKey = "x-ms-date" + private[storage] val XmsVersionHeaderKey = "x-ms-version" + private[storage] val BlobTypeHeaderKey = "x-ms-blob-type" + private[storage] val LeaseIdHeaderKey = "x-ms-lease-id" + private[storage] val FileWriteTypeHeaderKey = "x-ms-write" + private[storage] val PageWriteTypeHeaderKey = "x-ms-page-write" + private[storage] val XMsContentLengthHeaderKey = "x-ms-content-length" + private[storage] val FileTypeHeaderKey = "x-ms-type" + private[storage] val PageBlobContentLengthHeaderKey = "x-ms-blob-content-length" + private[storage] val PageBlobSequenceNumberHeaderKey = "x-ms-blob-sequence-number" + private[storage] val AnonymousAuthorizationType = "anon" + private[storage] val SharedKeyAuthorizationType = "SharedKey" + private[storage] val SasAuthorizationType = "sas" + private[storage] val BlobType = "blob" + private[storage] val FileType = "file" + private[storage] val BlockBlobType = "BlockBlob" + private[storage] val PageBlobType = "PageBlob" + private[storage] val AppendBlobType = "AppendBlob" + + private[storage] def getFormattedDate(implicit clock: Clock): String = + DateTimeFormatter.RFC_1123_DATE_TIME.format(clock.instant().atOffset(ZoneOffset.UTC)) + + /** Removes ETag quotes in the same way the official AWS tooling does. See + */ + private[storage] def removeQuotes(string: String): String = { + val trimmed = string.trim() + val tail = if (trimmed.startsWith("\"")) trimmed.drop(1) else trimmed + if (tail.endsWith("\"")) tail.dropRight(1) else tail + } + + /** This method returns `None` if given an empty `String`. This is typically used when parsing XML since its common to + * have XML elements with an empty text value inside. + */ + private[storage] def emptyStringToOption(value: String): Option[String] = if (value == "") None else Option(value) + + implicit private[storage] class ConfigOps(src: Config) { + + def getString(path: String, defaultValue: String): String = { + if (src.hasPath(path)) src.getString(path) else defaultValue + } + + def getOptionalString(path: String): Option[String] = + if (src.hasPath(path)) emptyStringToOption(src.getString(path)) else None + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/ClearFileRange.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/ClearFileRange.scala new file mode 100644 index 0000000000..ac546487f0 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/ClearFileRange.scala @@ -0,0 +1,59 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.headers.ByteRange +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.{RangeWriteTypeHeader, ServerSideEncryption} +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class ClearFileRange(val range: ByteRange.Slice, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("comp" -> "range") + + def withLeaseId(leaseId: String): ClearFileRange = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): ClearFileRange = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): ClearFileRange = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new ClearFileRange(range = range, leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentLengthHeader(0L) + .withRangeHeader(range) + .withLeaseIdHeader(leaseId) + .withRangeWriteTypeHeader(RangeWriteTypeHeader.ClearFileHeader) + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object ClearFileRange { + + /* + * Scala API + */ + def apply(range: ByteRange.Slice): ClearFileRange = new ClearFileRange(range) + + /* + * Java API + */ + def create(range: ByteRange.Slice): ClearFileRange = ClearFileRange(range) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateContainer.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateContainer.scala new file mode 100644 index 0000000000..c65226079d --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateContainer.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class CreateContainer(override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("restype" -> "container") + + override def withServerSideEncryption(sse: ServerSideEncryption): CreateContainer = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): CreateContainer = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(sse: Option[ServerSideEncryption] = sse, additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new CreateContainer(sse = sse, additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object CreateContainer { + + /* + * Scala API + */ + def apply(): CreateContainer = new CreateContainer() + + /* + * Java API + */ + def create(): CreateContainer = new CreateContainer() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateDirectory.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateDirectory.scala new file mode 100644 index 0000000000..78d5c2db72 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateDirectory.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class CreateDirectory(override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("restype" -> "directory") + + override def withServerSideEncryption(sse: ServerSideEncryption): CreateDirectory = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): CreateDirectory = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(sse: Option[ServerSideEncryption] = sse, additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new CreateDirectory(sse = sse, additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object CreateDirectory { + + /* + * Scala API + */ + def apply(): CreateDirectory = new CreateDirectory() + + /* + * Java API + */ + def create(): CreateDirectory = CreateDirectory() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateFile.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateFile.scala new file mode 100644 index 0000000000..7940208d2b --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/CreateFile.scala @@ -0,0 +1,63 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.{ContentType => JavaContentType} +import akka.http.scaladsl.model.{ContentType, HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class CreateFile(val maxFileSize: Long, + val contentType: ContentType, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + def withLeaseId(leaseId: String): CreateFile = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): CreateFile = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): CreateFile = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentTypeHeader(contentType) + .withFileMaxContentLengthHeader(maxFileSize) + .withFileTypeHeader() + .withLeaseIdHeader(leaseId) + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new CreateFile(maxFileSize = maxFileSize, + contentType = contentType, + leaseId = leaseId, + sse = sse, + additionalHeaders = additionalHeaders) +} + +object CreateFile { + + /* + * Scala API + */ + def apply(maxFileSize: Long, contentType: ContentType): CreateFile = new CreateFile(maxFileSize, contentType) + + /* + * Java API + */ + def create(maxFileSize: Long, contentType: JavaContentType): CreateFile = + CreateFile(maxFileSize, contentType.asInstanceOf[ContentType]) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteBlob.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteBlob.scala new file mode 100644 index 0000000000..dd85afb87b --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteBlob.scala @@ -0,0 +1,59 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class DeleteBlob(val versionId: Option[String] = None, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.DELETE + + override protected val queryParams: Map[String, String] = super.queryParams ++ + versionId.map(value => "versionId" -> value).toMap + + def withVersionId(versionId: String): DeleteBlob = copy(versionId = Option(versionId)) + + def withLeaseId(leaseId: String): DeleteBlob = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): DeleteBlob = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): DeleteBlob = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(versionId: Option[String] = versionId, + leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new DeleteBlob(versionId = versionId, leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) +} + +object DeleteBlob { + + /* + * Scala API + */ + def apply(): DeleteBlob = new DeleteBlob() + + /* + * Java API + */ + def create(): DeleteBlob = new DeleteBlob() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteContainer.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteContainer.scala new file mode 100644 index 0000000000..4bd931124c --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteContainer.scala @@ -0,0 +1,54 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class DeleteContainer(val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.DELETE + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("restype" -> "container") + + def withLeaseId(leaseId: String): DeleteContainer = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): DeleteContainer = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): DeleteContainer = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new DeleteContainer(leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) +} + +object DeleteContainer { + + /* + * Scala API + */ + def apply(): DeleteContainer = new DeleteContainer() + + /* + * Java API + */ + def create(): DeleteContainer = DeleteContainer() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteDirectory.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteDirectory.scala new file mode 100644 index 0000000000..b44efaff9f --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteDirectory.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class DeleteDirectory(override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.DELETE + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("restype" -> "directory") + + override def withServerSideEncryption(sse: ServerSideEncryption): DeleteDirectory = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): DeleteDirectory = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(sse: Option[ServerSideEncryption] = sse, additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new DeleteDirectory(sse = sse, additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object DeleteDirectory { + + /* + * Scala API + */ + def apply(): DeleteDirectory = new DeleteDirectory() + + /* + * Java API + */ + def create(): DeleteDirectory = DeleteDirectory() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteFile.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteFile.scala new file mode 100644 index 0000000000..d89ffb337b --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/DeleteFile.scala @@ -0,0 +1,59 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class DeleteFile(val versionId: Option[String] = None, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.DELETE + + override protected val queryParams: Map[String, String] = super.queryParams ++ + versionId.map(value => "versionId" -> value).toMap + + def withVersionId(versionId: String): DeleteFile = copy(versionId = Option(versionId)) + + def withLeaseId(leaseId: String): DeleteFile = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): DeleteFile = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): DeleteFile = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(versionId: Option[String] = versionId, + leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new DeleteFile(versionId = versionId, leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) +} + +object DeleteFile { + + /* + * Scala API + */ + def apply(): DeleteFile = new DeleteFile() + + /* + * Java API + */ + def create(): DeleteFile = new DeleteFile() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetBlob.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetBlob.scala new file mode 100644 index 0000000000..7cb8643779 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetBlob.scala @@ -0,0 +1,74 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.headers.{ByteRange => JavaByteRange} +import akka.http.scaladsl.model.headers.ByteRange +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class GetBlob(val versionId: Option[String] = None, + val range: Option[ByteRange] = None, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.GET + + override protected val queryParams: Map[String, String] = super.queryParams ++ + versionId.map(value => "versionId" -> value).toMap + + def withVersionId(versionId: String): GetBlob = copy(versionId = Option(versionId)) + + /** Java API */ + def withRange(range: JavaByteRange): GetBlob = withRange(range.asInstanceOf[ByteRange]) + + /** Scala API */ + def withRange(range: ByteRange): GetBlob = copy(range = Option(range)) + + def withLeaseId(leaseId: String): GetBlob = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): GetBlob = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): GetBlob = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(versionId: Option[String] = versionId, + range: Option[ByteRange] = range, + leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new GetBlob(versionId = versionId, + range = range, + leaseId = leaseId, + sse = sse, + additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withRangeHeader(range) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object GetBlob { + + /* + * Scala API + */ + def apply(): GetBlob = new GetBlob() + + /* + * Java API + */ + def create(): GetBlob = new GetBlob() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetFile.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetFile.scala new file mode 100644 index 0000000000..16289ef356 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetFile.scala @@ -0,0 +1,74 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.headers.{ByteRange => JavaByteRange} +import akka.http.scaladsl.model.headers.ByteRange +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class GetFile(val versionId: Option[String] = None, + val range: Option[ByteRange] = None, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.GET + + override protected val queryParams: Map[String, String] = super.queryParams ++ + versionId.map(value => "versionId" -> value).toMap + + def withVersionId(versionId: String): GetFile = copy(versionId = Option(versionId)) + + /** Java API */ + def withRange(range: JavaByteRange): GetFile = withRange(range.asInstanceOf[ByteRange]) + + /** Scala API */ + def withRange(range: ByteRange): GetFile = copy(range = Option(range)) + + def withLeaseId(leaseId: String): GetFile = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): GetFile = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): GetFile = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(versionId: Option[String] = versionId, + range: Option[ByteRange] = range, + leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new GetFile(versionId = versionId, + range = range, + leaseId = leaseId, + sse = sse, + additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withRangeHeader(range) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object GetFile { + + /* + * Scala API + */ + def apply(): GetFile = new GetFile() + + /* + * Java API + */ + def create(): GetFile = new GetFile() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetProperties.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetProperties.scala new file mode 100644 index 0000000000..b56d3d6a7d --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/GetProperties.scala @@ -0,0 +1,59 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class GetProperties(val versionId: Option[String] = None, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.HEAD + + override protected val queryParams: Map[String, String] = super.queryParams ++ + versionId.map(value => "versionId" -> value).toMap + + def withVersionId(versionId: String): GetProperties = copy(versionId = Option(versionId)) + + def withLeaseId(leaseId: String): GetProperties = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): GetProperties = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): GetProperties = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .witServerSideEncryption(sse) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(versionId: Option[String] = versionId, + leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new GetProperties(versionId = versionId, leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) +} + +object GetProperties { + + /* + * Scala API + */ + def apply(): GetProperties = new GetProperties() + + /* + * Java API + */ + def create(): GetProperties = new GetProperties() +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutAppendBlock.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutAppendBlock.scala new file mode 100644 index 0000000000..57a64fda03 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutAppendBlock.scala @@ -0,0 +1,57 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.{ContentType => JavaContentType} +import akka.http.scaladsl.model.{ContentType, HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.{BlobTypeHeader, ServerSideEncryption} +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class PutAppendBlock(val contentType: ContentType, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + def withLeaseId(leaseId: String): PutAppendBlock = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): PutAppendBlock = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): PutAppendBlock = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentLengthHeader(0L) + .withContentTypeHeader(contentType) + .witServerSideEncryption(sse) + .withBlobTypeHeader(BlobTypeHeader.PageBlobHeader) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new PutAppendBlock(contentType = contentType, leaseId = leaseId, sse = sse, additionalHeaders = additionalHeaders) +} + +object PutAppendBlock { + + /* + * Scala API + */ + def apply(contentType: ContentType): PutAppendBlock = new PutAppendBlock(contentType) + + /* + * Java API + */ + def create(contentType: JavaContentType): PutAppendBlock = PutAppendBlock(contentType.asInstanceOf[ContentType]) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutBlockBlob.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutBlockBlob.scala new file mode 100644 index 0000000000..f7647ae46f --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutBlockBlob.scala @@ -0,0 +1,63 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.{ContentType => JavaContentType} +import akka.http.scaladsl.model.{ContentType, HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.{BlobTypeHeader, ServerSideEncryption} +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class PutBlockBlob(val contentLength: Long, + val contentType: ContentType, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + def withLeaseId(leaseId: String): PutBlockBlob = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): PutBlockBlob = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): PutBlockBlob = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentLengthHeader(contentLength) + .withContentTypeHeader(contentType) + .witServerSideEncryption(sse) + .withBlobTypeHeader(BlobTypeHeader.BlockBlobHeader) + .withLeaseIdHeader(leaseId) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new PutBlockBlob(contentLength = contentLength, + contentType = contentType, + leaseId = leaseId, + sse = sse, + additionalHeaders = additionalHeaders) +} + +object PutBlockBlob { + + /* + * Scala API + */ + def apply(contentLength: Long, contentType: ContentType): PutBlockBlob = new PutBlockBlob(contentLength, contentType) + + /* + * Java API + */ + def create(contentLength: Long, contentType: JavaContentType): PutBlockBlob = + PutBlockBlob(contentLength, contentType.asInstanceOf[ContentType]) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutPageBlock.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutPageBlock.scala new file mode 100644 index 0000000000..4349fc5f46 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/PutPageBlock.scala @@ -0,0 +1,71 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.{ContentType => JavaContentType} +import akka.http.scaladsl.model.{ContentType, HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.{BlobTypeHeader, ServerSideEncryption} +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class PutPageBlock(val maxBlockSize: Long, + val contentType: ContentType, + val leaseId: Option[String] = None, + val blobSequenceNumber: Option[Int] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + def withLeaseId(leaseId: String): PutPageBlock = copy(leaseId = Option(leaseId)) + + def withBlobSequenceNumber(blobSequenceNumber: Int): PutPageBlock = + copy(blobSequenceNumber = Option(blobSequenceNumber)) + + override def withServerSideEncryption(sse: ServerSideEncryption): PutPageBlock = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): PutPageBlock = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentLengthHeader(0L) + .withPageBlobContentLengthHeader(maxBlockSize) + .withContentTypeHeader(contentType) + .witServerSideEncryption(sse) + .withBlobTypeHeader(BlobTypeHeader.PageBlobHeader) + .withLeaseIdHeader(leaseId) + .withPageBlobSequenceNumberHeader(blobSequenceNumber) + .withAdditionalHeaders(additionalHeaders) + .headers + + private def copy(leaseId: Option[String] = leaseId, + blobSequenceNumber: Option[Int] = blobSequenceNumber, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new PutPageBlock(maxBlockSize = maxBlockSize, + contentType = contentType, + leaseId = leaseId, + blobSequenceNumber = blobSequenceNumber, + sse = sse, + additionalHeaders = additionalHeaders) +} + +object PutPageBlock { + + /* + * Scala API + */ + def apply(maxBlockSize: Long, contentType: ContentType): PutPageBlock = new PutPageBlock(maxBlockSize, contentType) + + /* + * Java API + */ + def create(maxBlockSize: Long, contentType: JavaContentType): PutPageBlock = + new PutPageBlock(maxBlockSize, contentType.asInstanceOf[ContentType]) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/RequestBuilder.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/RequestBuilder.scala new file mode 100644 index 0000000000..a492224342 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/RequestBuilder.scala @@ -0,0 +1,72 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.scaladsl.model.Uri.Query +import akka.http.scaladsl.model.headers.RawHeader +import akka.http.scaladsl.model.{HttpHeader, HttpMethod, HttpRequest, Uri} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption + +abstract class RequestBuilder(val sse: Option[ServerSideEncryption] = None, + val additionalHeaders: Seq[HttpHeader] = Seq.empty) { + + protected val method: HttpMethod + + protected def queryParams: Map[String, String] = Map.empty + + def withServerSideEncryption(sse: ServerSideEncryption): RequestBuilder + + def addHeader(httpHeader: HttpHeader): RequestBuilder + + def addHeader(name: String, value: String): RequestBuilder = addHeader(RawHeader(name, value)) + + protected def getHeaders: Seq[HttpHeader] + + private[storage] def createRequest(settings: StorageSettings, storageType: String, objectPath: String): HttpRequest = + HttpRequest( + method = method, + uri = createUri( + settings = settings, + storageType = storageType, + objectPath = objectPath, + queryString = createQueryString(settings, emptyStringToOption(Query(queryParams).toString())) + ), + headers = getHeaders + ) + + private def createUri(settings: StorageSettings, + storageType: String, + objectPath: String, + queryString: Option[String]): Uri = { + val accountName = settings.azureNameKeyCredential.accountName + val path = if (objectPath.startsWith("/")) objectPath else s"/$objectPath" + settings.endPointUrl + .map { endPointUrl => + val qs = queryString.getOrElse("") + Uri(endPointUrl).withPath(Uri.Path(s"/$accountName$path")).withQuery(Uri.Query(qs)) + } + .getOrElse( + Uri.from( + scheme = "https", + host = s"$accountName.$storageType.core.windows.net", + path = Uri.Path(path).toString(), + queryString = queryString + ) + ) + } + + private def createQueryString(settings: StorageSettings, apiQueryString: Option[String]): Option[String] = { + if (settings.authorizationType == SasAuthorizationType) { + if (settings.sasToken.isEmpty) throw new RuntimeException("SAS token must be defined for SAS authorization type.") + else { + val sasToken = settings.sasToken.get + Some(apiQueryString.map(qs => s"$sasToken&$qs").getOrElse(sasToken)) + } + } else apiQueryString + } +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/UpdateFileRange.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/UpdateFileRange.scala new file mode 100644 index 0000000000..e71ca27bc0 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/UpdateFileRange.scala @@ -0,0 +1,67 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package requests + +import akka.http.javadsl.model.{ContentType => JavaContentType} +import akka.http.scaladsl.model.headers.ByteRange +import akka.http.scaladsl.model.{ContentType, HttpHeader, HttpMethod, HttpMethods} +import akka.stream.alpakka.azure.storage.headers.{RangeWriteTypeHeader, ServerSideEncryption} +import akka.stream.alpakka.azure.storage.impl.StorageHeaders + +final class UpdateFileRange(val range: ByteRange.Slice, + val contentType: ContentType, + val leaseId: Option[String] = None, + override val sse: Option[ServerSideEncryption] = None, + override val additionalHeaders: Seq[HttpHeader] = Seq.empty) + extends RequestBuilder { + + override protected val method: HttpMethod = HttpMethods.PUT + + override protected val queryParams: Map[String, String] = super.queryParams ++ Map("comp" -> "range") + + def withLeaseId(leaseId: String): UpdateFileRange = copy(leaseId = Option(leaseId)) + + override def withServerSideEncryption(sse: ServerSideEncryption): UpdateFileRange = copy(sse = Option(sse)) + + override def addHeader(httpHeader: HttpHeader): UpdateFileRange = + copy(additionalHeaders = additionalHeaders :+ httpHeader) + + private def copy(leaseId: Option[String] = leaseId, + sse: Option[ServerSideEncryption] = sse, + additionalHeaders: Seq[HttpHeader] = additionalHeaders) = + new UpdateFileRange(contentType = contentType, + range = range, + leaseId = leaseId, + sse = sse, + additionalHeaders = additionalHeaders) + + override protected def getHeaders: Seq[HttpHeader] = + StorageHeaders() + .withContentLengthHeader(range.last - range.first + 1) + .withContentTypeHeader(contentType) + .withRangeHeader(range) + .withLeaseIdHeader(leaseId) + .withRangeWriteTypeHeader(RangeWriteTypeHeader.UpdateFileHeader) + .witServerSideEncryption(sse) + .withAdditionalHeaders(additionalHeaders) + .headers +} + +object UpdateFileRange { + + /* + * Scala API + */ + def apply(range: ByteRange.Slice, contentType: ContentType): UpdateFileRange = new UpdateFileRange(range, contentType) + + /* + * Java API + */ + def create(range: ByteRange.Slice, contentType: JavaContentType): UpdateFileRange = + UpdateFileRange(range, contentType.asInstanceOf[ContentType]) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/BlobService.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/BlobService.scala new file mode 100644 index 0000000000..8eceac1939 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/BlobService.scala @@ -0,0 +1,125 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.NotUsed +import akka.http.scaladsl.model.HttpEntity +import akka.stream.alpakka.azure.storage.impl.AzureStorageStream +import akka.stream.alpakka.azure.storage.requests.{ + CreateContainer, + DeleteBlob, + DeleteContainer, + GetBlob, + GetProperties, + PutAppendBlock, + PutBlockBlob, + PutPageBlock +} +import akka.stream.scaladsl.Source +import akka.util.ByteString + +import scala.concurrent.Future + +/** + * Scala API for BlobService operations. + */ +object BlobService { + + /** + * Gets blob representing `objectPath` with specified range (if applicable). + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build getBlob request + * @return A [[akka.stream.scaladsl.Source]] containing the objects data as a [[akka.util.ByteString]] along with a + * materialized value containing the [[akka.stream.alpakka.azure.storage.ObjectMetadata]] + */ + def getBlob(objectPath: String, requestBuilder: GetBlob): Source[ByteString, Future[ObjectMetadata]] = + AzureStorageStream.getObject(BlobType, objectPath, requestBuilder) + + /** + * Gets blob properties. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder versionId of the blob (if applicable) + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def getProperties(objectPath: String, requestBuilder: GetProperties): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.getBlobProperties(objectPath, requestBuilder) + + /** + * Deletes blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build deleteBlob request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteBlob(objectPath: String, requestBuilder: DeleteBlob): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteBlob(objectPath, requestBuilder) + + /** + * Put Block blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build putBlockBlob request + * @param payload actual payload, a [[akka.stream.scaladsl.Source Source]] of [[akka.util.ByteString ByteString]] + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putBlockBlob(objectPath: String, + requestBuilder: PutBlockBlob, + payload: Source[ByteString, _]): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream + .putBlob(objectPath, + requestBuilder, + Some(HttpEntity(requestBuilder.contentType, requestBuilder.contentLength, payload))) + + /** + * Put (Create) Page Blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @param requestBuilder builder to build putAppendBlob request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putPageBlock(objectPath: String, requestBuilder: PutPageBlock): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.putBlob(objectPath, requestBuilder, None) + + /** + * Put (Create) Append Blob. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/container/blob` + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def putAppendBlock(objectPath: String, requestBuilder: PutAppendBlock): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.putBlob(objectPath, requestBuilder, None) + + /** + * Create container. + * + * @param objectPath name of the container + * @param requestBuilder builder to build createContainer request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createContainer(objectPath: String, requestBuilder: CreateContainer): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.createContainer(objectPath, requestBuilder) + + /** + * Delete container. + * + * @param objectPath name of the container + * @param requestBuilder builder to build deleteContainer request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteContainer(objectPath: String, requestBuilder: DeleteContainer): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteContainer(objectPath, requestBuilder) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/FileService.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/FileService.scala new file mode 100644 index 0000000000..2f61889745 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/scaladsl/FileService.scala @@ -0,0 +1,128 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.NotUsed +import akka.http.scaladsl.model.HttpEntity +import akka.stream.alpakka.azure.storage.impl.AzureStorageStream +import akka.stream.alpakka.azure.storage.requests.{ + ClearFileRange, + CreateDirectory, + CreateFile, + DeleteDirectory, + DeleteFile, + GetFile, + GetProperties, + UpdateFileRange +} +import akka.stream.scaladsl.Source +import akka.util.ByteString + +import scala.concurrent.Future + +/** + * Scala API for FileService operations. + */ +object FileService { + + /** + * Gets file representing `objectPath` with specified range (if applicable). + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build getFile request + * @return A [[akka.stream.scaladsl.Source]] containing the objects data as a [[akka.util.ByteString]] along with a + * materialized value containing the [[akka.stream.alpakka.azure.storage.ObjectMetadata]] + */ + def getFile(objectPath: String, requestBuilder: GetFile): Source[ByteString, Future[ObjectMetadata]] = + AzureStorageStream.getObject(FileType, objectPath, requestBuilder) + + /** + * Gets file properties. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build getFile properties request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def getProperties(objectPath: String, requestBuilder: GetProperties): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.getFileProperties(objectPath, requestBuilder) + + /** + * Deletes file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build deleteFile request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteFile(objectPath: String, requestBuilder: DeleteFile): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteFile(objectPath, requestBuilder) + + /** + * Creates a file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build createFile request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createFile(objectPath: String, requestBuilder: CreateFile): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.createFile(objectPath, requestBuilder) + + /** + * Updates file on the specified range. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build updateRange request + * @param payload actual payload, a [[akka.stream.scaladsl.Source Source]] of [[akka.util.ByteString ByteString]] + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def updateRange(objectPath: String, + requestBuilder: UpdateFileRange, + payload: Source[ByteString, _]): Source[Option[ObjectMetadata], NotUsed] = { + AzureStorageStream.updateRange( + objectPath, + HttpEntity(requestBuilder.contentType, requestBuilder.range.last - requestBuilder.range.first + 1, payload), + requestBuilder + ) + } + + /** + * Clears specified range from the file. + * + * @param objectPath path of the object, should start with "/" and separated by `/`, e.g. `/share/my-directory/blob` + * @param requestBuilder builder to build clearRange request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def clearRange(objectPath: String, requestBuilder: ClearFileRange): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.clearRange(objectPath, requestBuilder) + + /** + * Create directory. + * + * @param directoryPath path of the directory to be created, e.g., `myshare/myparentdirectorypath/mydirectory` + * @param requestBuilder builder to build createDirectory request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def createDirectory(directoryPath: String, requestBuilder: CreateDirectory): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.createDirectory(directoryPath, requestBuilder) + + /** + * Delete directory. + * + * @param directoryPath path of the directory to be deleted, e.g., `myshare/myparentdirectorypath/mydirectory` + * @param requestBuilder builder to build deleteDirectory request + * @return A [[akka.stream.scaladsl.Source Source]] containing an [[scala.Option]] of + * [[akka.stream.alpakka.azure.storage.ObjectMetadata]], will be [[scala.None]] in case the object does not exist + */ + def deleteDirectory(directoryPath: String, requestBuilder: DeleteDirectory): Source[Option[ObjectMetadata], NotUsed] = + AzureStorageStream.deleteDirectory(directoryPath, requestBuilder) +} diff --git a/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/settings.scala b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/settings.scala new file mode 100644 index 0000000000..45fcde8ff6 --- /dev/null +++ b/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/settings.scala @@ -0,0 +1,277 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.actor.ClassicActorSystemProvider +import com.typesafe.config.Config + +import java.time.{Duration => JavaDuration} +import java.util.{Objects, Optional} +import java.util.concurrent.TimeUnit +import scala.concurrent.duration._ +import scala.jdk.OptionConverters._ +import scala.util.Try + +final class StorageSettings(val apiVersion: String, + val authorizationType: String, + val endPointUrl: Option[String], + val azureNameKeyCredential: AzureNameKeyCredential, + val sasToken: Option[String], + val retrySettings: RetrySettings, + val algorithm: String) { + + /** Java API */ + def getApiVersion: String = apiVersion + + /** Java API */ + def getAuthorizationType: String = authorizationType + + /** Java API */ + def getEndPointUrl: Optional[String] = endPointUrl.toJava + + /** Java API */ + def getAzureNameKeyCredential: AzureNameKeyCredential = azureNameKeyCredential + + /** Java API */ + def getSasToken: Optional[String] = sasToken.toJava + + /** Java API */ + def getRetrySettings: RetrySettings = retrySettings + + /** Java API */ + def getAlgorithm: String = algorithm + + /** Java API */ + def witApiVersion(apiVersion: String): StorageSettings = copy(apiVersion = apiVersion) + + /** Java API */ + def withAuthorizationType(authorizationType: String): StorageSettings = copy(authorizationType = authorizationType) + + /** Java API */ + def withSasToken(sasToken: String): StorageSettings = copy(sasToken = emptyStringToOption(sasToken)) + + /** Java API */ + def withAzureNameKeyCredential(azureNameKeyCredential: AzureNameKeyCredential): StorageSettings = + copy(azureNameKeyCredential = azureNameKeyCredential) + + /** Java API */ + def withEndPointUrl(endPointUrl: String): StorageSettings = copy(endPointUrl = emptyStringToOption(endPointUrl)) + + /** Java API */ + def withRetrySettings(retrySettings: RetrySettings): StorageSettings = copy(retrySettings = retrySettings) + + /** Java API */ + def withAlgorithm(algorithm: String): StorageSettings = copy(algorithm = algorithm) + + override def toString: String = + s"""StorageSettings( + | apiVersion=$apiVersion, + | authorizationType=$authorizationType, + | endPointUrl=$endPointUrl, + | azureNameKeyCredential=$azureNameKeyCredential, + | sasToken=$sasToken + | retrySettings=$retrySettings, + | algorithm=$algorithm + |)""".stripMargin.replaceAll(System.lineSeparator(), "") + + override def equals(other: Any): Boolean = other match { + case that: StorageSettings => + apiVersion == that.apiVersion && + authorizationType == that.authorizationType && + endPointUrl == that.endPointUrl && + Objects.equals(azureNameKeyCredential, that.azureNameKeyCredential) && + sasToken == that.sasToken && + Objects.equals(retrySettings, that.retrySettings) && + algorithm == that.algorithm + + case _ => false + } + + override def hashCode(): Int = + Objects.hash(apiVersion, authorizationType, azureNameKeyCredential, sasToken, retrySettings, algorithm) + + private def copy( + apiVersion: String = apiVersion, + authorizationType: String = authorizationType, + endPointUrl: Option[String] = endPointUrl, + azureNameKeyCredential: AzureNameKeyCredential = azureNameKeyCredential, + sasToken: Option[String] = sasToken, + retrySettings: RetrySettings = retrySettings, + algorithm: String = algorithm + ) = + StorageSettings(apiVersion, + authorizationType, + endPointUrl, + azureNameKeyCredential, + sasToken, + retrySettings, + algorithm) +} + +object StorageSettings { + private[storage] val ConfigPath = "alpakka.azure-storage" + private val AuthorizationTypes = Seq(AnonymousAuthorizationType, SharedKeyAuthorizationType, SasAuthorizationType) + + def apply( + apiVersion: String, + authorizationType: String, + endPointUrl: Option[String], + azureNameKeyCredential: AzureNameKeyCredential, + sasToken: Option[String], + retrySettings: RetrySettings, + algorithm: String + ): StorageSettings = + new StorageSettings(apiVersion, + authorizationType, + endPointUrl, + azureNameKeyCredential, + sasToken, + retrySettings, + algorithm) + + /** Java API */ + def create( + apiVersion: String, + authorizationType: String, + endPointUrl: Optional[String], + azureNameKeyCredential: AzureNameKeyCredential, + sasToken: Optional[String], + retrySettings: RetrySettings, + algorithm: String + ): StorageSettings = + StorageSettings(apiVersion, + authorizationType, + Option(endPointUrl.orElse(null)), + azureNameKeyCredential, + Option(sasToken.orElse(null)), + retrySettings, + algorithm) + + def apply(config: Config): StorageSettings = { + val apiVersion = config.getString("api-version", "2024-11-04") + + val credentials = + if (config.hasPath("credentials")) config.getConfig("credentials") + else throw new RuntimeException("credentials must be defined.") + + val authorizationType = { + val value = credentials.getString("authorization-type", "anon") + if (AuthorizationTypes.contains(value)) value else AnonymousAuthorizationType + } + + val retrySettings = + if (config.hasPath("retry-settings")) RetrySettings(config.getConfig("retry-settings")) else RetrySettings.Default + + StorageSettings( + apiVersion = apiVersion, + authorizationType = authorizationType, + endPointUrl = config.getOptionalString("endpoint-url"), + azureNameKeyCredential = AzureNameKeyCredential(credentials), + sasToken = credentials.getOptionalString("sas-token"), + retrySettings = retrySettings, + algorithm = config.getString("signing-algorithm", "HmacSHA256") + ) + } + + def apply()(implicit system: ClassicActorSystemProvider): StorageSettings = + StorageSettings(system.classicSystem.settings.config.getConfig(ConfigPath)) +} + +final class RetrySettings private (val maxRetries: Int, + val minBackoff: FiniteDuration, + val maxBackoff: FiniteDuration, + val randomFactor: Double) { + + /** Java API */ + def getMaxRetries: Int = maxRetries + + /** Java API */ + def getMinBackoff: JavaDuration = JavaDuration.ofNanos(minBackoff.toNanos) + + /** Java API */ + def getMaxBackoff: JavaDuration = JavaDuration.ofNanos(maxBackoff.toNanos) + + /** Java API */ + def getRandomFactor: Double = randomFactor + + def withMaxRetries(value: Int): RetrySettings = copy(maxRetries = value) + + def withMinBackoff(value: FiniteDuration): RetrySettings = copy(minBackoff = value) + + /** Java API */ + def withMinBackoff(value: JavaDuration): RetrySettings = + copy(minBackoff = FiniteDuration(value.toNanos, TimeUnit.NANOSECONDS)) + + def withMaxBackoff(value: FiniteDuration): RetrySettings = copy(maxBackoff = value) + + /** Java API */ + def withMaxBackoff(value: JavaDuration): RetrySettings = + copy(maxBackoff = FiniteDuration(value.toNanos, TimeUnit.NANOSECONDS)) + + def withRandomFactor(value: Double): RetrySettings = copy(randomFactor = value) + + private def copy( + maxRetries: Int = maxRetries, + minBackoff: FiniteDuration = minBackoff, + maxBackoff: FiniteDuration = maxBackoff, + randomFactor: Double = randomFactor + ) = + new RetrySettings(maxRetries, minBackoff, maxBackoff, randomFactor) + + override def toString: String = + "RetrySettings(" + + s"maxRetries=$maxRetries," + + s"minBackoff=$minBackoff," + + s"maxBackoff=$maxBackoff," + + s"randomFactor=$randomFactor)" + + override def equals(other: Any): Boolean = other match { + case that: RetrySettings => + Objects.equals(this.maxRetries, that.maxRetries) && + Objects.equals(this.minBackoff, that.minBackoff) && + Objects.equals(this.maxBackoff, that.maxBackoff) && + Objects.equals(this.randomFactor, that.randomFactor) + case _ => false + } + + override def hashCode(): Int = + Objects.hash(Int.box(maxRetries), minBackoff, maxBackoff, Double.box(randomFactor)) +} + +object RetrySettings { + val Default: RetrySettings = RetrySettings(3, 200.milliseconds, 10.seconds, 0.0) + + /** Scala API */ + def apply( + maxRetries: Int, + minBackoff: FiniteDuration, + maxBackoff: FiniteDuration, + randomFactor: Double + ): RetrySettings = + new RetrySettings(maxRetries, minBackoff, maxBackoff, randomFactor) + + /** Java API */ + def create(maxRetries: Int, minBackoff: JavaDuration, maxBackoff: JavaDuration, randomFactor: Double): RetrySettings = + apply( + maxRetries, + FiniteDuration(minBackoff.toNanos, TimeUnit.NANOSECONDS), + FiniteDuration(maxBackoff.toNanos, TimeUnit.NANOSECONDS), + randomFactor + ) + + def apply(config: Config): RetrySettings = { + Try( + RetrySettings( + config.getInt("max-retries"), + FiniteDuration(config.getDuration("min-backoff").toNanos, TimeUnit.NANOSECONDS), + FiniteDuration(config.getDuration("max-backoff").toNanos, TimeUnit.NANOSECONDS), + config.getDouble("random-factor") + ) + ).toOption.getOrElse(Default) + } + +} diff --git a/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java b/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java new file mode 100644 index 0000000000..72d723beca --- /dev/null +++ b/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java @@ -0,0 +1,83 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package docs.javadsl; + +import akka.http.javadsl.model.ContentTypes; +import akka.http.javadsl.model.headers.ByteRange; +import akka.http.scaladsl.model.headers.RawHeader; +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption; +import akka.stream.alpakka.azure.storage.requests.CreateFile; +import akka.stream.alpakka.azure.storage.requests.GetBlob; +import akka.stream.alpakka.azure.storage.requests.PutBlockBlob; +import akka.stream.alpakka.testkit.javadsl.LogCapturingJunit4; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import scala.Option; + +public class RequestBuilderTest { + + @Rule + public final LogCapturingJunit4 logCapturing = new LogCapturingJunit4(); + + @Test + public void createSimpleRequest() { + + //#simple-request-builder + final GetBlob requestBuilder = GetBlob.create(); + //#simple-request-builder + + Assert.assertEquals(Option.empty(), requestBuilder.versionId()); + Assert.assertEquals(Option.empty(), requestBuilder.range()); + Assert.assertEquals(Option.empty(), requestBuilder.leaseId()); + Assert.assertEquals(Option.empty(), requestBuilder.sse()); + } + + @Test + public void populateRequestBuilder() { + + //#populate-request-builder + final var requestBuilder = GetBlob.create().withLeaseId("my-lease-id").withRange(ByteRange.createSlice(0, 25)); + //#populate-request-builder + + Assert.assertEquals(Option.apply("my-lease-id"), requestBuilder.leaseId()); + Assert.assertEquals(Option.apply(ByteRange.createSlice(0, 25)), requestBuilder.range()); + Assert.assertEquals(Option.empty(), requestBuilder.sse()); + } + + @Test + public void createRequestBuilderWithMandatoryParams() { + + //#request-builder-with-initial-values + final var requestBuilder = CreateFile.create(256L, ContentTypes.TEXT_PLAIN_UTF8); + //#request-builder-with-initial-values + + Assert.assertEquals(Option.empty(), requestBuilder.leaseId()); + Assert.assertEquals(256L, requestBuilder.maxFileSize()); + Assert.assertEquals(ContentTypes.TEXT_PLAIN_UTF8, requestBuilder.contentType()); + } + + @Test + public void populateServerSideEncryption() { + + //#request-builder-with-sse + final var requestBuilder = PutBlockBlob.create(256L, ContentTypes.TEXT_PLAIN_UTF8) + .withServerSideEncryption(ServerSideEncryption.customerKey("SGVsbG9Xb3JsZA==")); + //#request-builder-with-sse + + Assert.assertEquals(ServerSideEncryption.customerKey("SGVsbG9Xb3JsZA=="), requestBuilder.sse().get()); + } + + @Test + public void populateAdditionalHeaders() { + + //#request-builder-with-additional-headers + final var requestBuilder = GetBlob.create().addHeader("If-Match", "foobar"); + //#request-builder-with-additional-headers + + Assert.assertEquals(1, requestBuilder.additionalHeaders().size()); + Assert.assertEquals(new RawHeader("If-Match", "foobar"), requestBuilder.additionalHeaders().head()); + } +} diff --git a/azure-storage/src/test/java/docs/javadsl/StorageTest.java b/azure-storage/src/test/java/docs/javadsl/StorageTest.java new file mode 100644 index 0000000000..68cdca48fb --- /dev/null +++ b/azure-storage/src/test/java/docs/javadsl/StorageTest.java @@ -0,0 +1,377 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package docs.javadsl; + +import akka.NotUsed; +import akka.actor.ActorSystem; +import akka.http.javadsl.Http; +import akka.http.javadsl.model.ContentTypes; +import akka.stream.alpakka.azure.storage.ObjectMetadata; +import akka.stream.alpakka.azure.storage.javadsl.BlobService; +import akka.stream.alpakka.azure.storage.javadsl.FileService; +import akka.stream.alpakka.azure.storage.requests.ClearFileRange; +import akka.stream.alpakka.azure.storage.requests.CreateContainer; +import akka.stream.alpakka.azure.storage.requests.CreateDirectory; +import akka.stream.alpakka.azure.storage.requests.CreateFile; +import akka.stream.alpakka.azure.storage.requests.DeleteContainer; +import akka.stream.alpakka.azure.storage.requests.DeleteDirectory; +import akka.stream.alpakka.azure.storage.requests.DeleteFile; +import akka.stream.alpakka.azure.storage.requests.GetBlob; +import akka.stream.alpakka.azure.storage.requests.GetFile; +import akka.stream.alpakka.azure.storage.requests.GetProperties; +import akka.stream.alpakka.azure.storage.requests.PutAppendBlock; +import akka.stream.alpakka.azure.storage.requests.PutBlockBlob; +import akka.stream.alpakka.azure.storage.requests.PutPageBlock; +import akka.stream.alpakka.azure.storage.requests.UpdateFileRange; +import akka.stream.alpakka.azure.storage.scaladsl.StorageWireMockBase; +import akka.stream.alpakka.testkit.javadsl.LogCapturingJunit4; +import akka.stream.javadsl.Source; +import akka.stream.javadsl.Sink; +import akka.testkit.javadsl.TestKit; +import akka.util.ByteString; +import com.github.tomakehurst.wiremock.WireMockServer; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import scala.Option; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletionStage; +import java.util.stream.Collectors; + +public class StorageTest extends StorageWireMockBase { + + @Rule + public final LogCapturingJunit4 logCapturing = new LogCapturingJunit4(); + + private static ActorSystem system; + private static WireMockServer wireMockServerForShutdown; + + @Before + public void before() { + wireMockServerForShutdown = _wireMockServer(); + system = system(); + } + + @AfterClass + public static void afterAll() { + wireMockServerForShutdown.stop(); + Http.get(system) + .shutdownAllConnectionPools() + .thenRun(() -> TestKit.shutdownActorSystem(system)); + } + + @Test + public void createContainer() throws Exception { + mockCreateContainer(); + + //#create-container + final Source, NotUsed> source = BlobService.createContainer(containerName(), CreateContainer.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#create-container + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + final var objectMetadata = optionalObjectMetadata.get(); + Assert.assertEquals(objectMetadata.getContentLength(), 0L); + Assert.assertEquals(objectMetadata.getETag().get(), ETagRawValue()); + } + + @Test + public void deleteContainer() throws Exception { + mockDeleteContainer(); + + //#delete-container + final Source, NotUsed> source = BlobService.deleteContainer(containerName(), DeleteContainer.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#delete-container + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + final var objectMetadata = optionalObjectMetadata.get(); + Assert.assertEquals(objectMetadata.getContentLength(), 0L); + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + @Ignore("Fix this test case") + @Test + public void putBlockBlob() throws Exception { + mockPutBlockBlob(); + + //#put-block-blob + final Source, NotUsed> source = + BlobService.putBlockBlob(containerName() + "/" + blobName(), + PutBlockBlob.create(contentLength(), ContentTypes.TEXT_PLAIN_UTF8), + Source.single(ByteString.fromString(payload()))); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#put-block-blob + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + } + + @Ignore("Test is failing due to multiple content length headers in the request.") + @Test + public void putPageBlob() throws Exception { + mockPutPageBlob(); + + //#put-page-blob + final Source, NotUsed> source = + BlobService.putPageBlock(containerName() + "/" + blobName(), + PutPageBlock.create(512L, ContentTypes.TEXT_PLAIN_UTF8).withBlobSequenceNumber(0)); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#put-page-blob + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + } + + @Ignore("Test is failing due to multiple content length headers in the request.") + @Test + public void putAppendBlob() throws Exception { + mockPutAppendBlob(); + + //#put-append-blob + final Source, NotUsed> source = + BlobService.putAppendBlock(containerName() + "/" + blobName(), + PutAppendBlock.create(ContentTypes.TEXT_PLAIN_UTF8)); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#put-append-blob + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + } + + @Test + public void getBlob() throws Exception { + mockGetBlob(Option.empty(), Option.empty()); + + //#get-blob + final Source> source = + BlobService.getBlob(containerName() + "/" + blobName(), GetBlob.create()); + + final CompletionStage> eventualPayload = source.runWith(Sink.seq(), system); + //#get-blob + + final var actualPayload = eventualPayload.toCompletableFuture().get().stream() + .map(ByteString::utf8String).collect(Collectors.joining()); + Assert.assertEquals(actualPayload, payload()); + } + + @Test + public void getBlobRange() throws Exception { + mockGetBlobWithRange(); + + //#get-blob-range + final Source> source = + BlobService.getBlob(containerName() + "/" + blobName(), GetBlob.create().withRange(subRange())); + + final CompletionStage> eventualPayload = source.runWith(Sink.seq(), system); + //#get-blob-range + + final var actualPayload = eventualPayload.toCompletableFuture().get().stream() + .map(ByteString::utf8String).collect(Collectors.joining()); + Assert.assertEquals("quick", actualPayload); + } + + @Test + public void getBlobProperties() throws Exception { + mockGetBlobProperties(); + + //#get-blob-properties + final Source, NotUsed> source = + BlobService.getProperties(containerName() + "/" + blobName(), GetProperties.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#get-blob-properties + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(contentLength(), objectMetadata.getContentLength()); + Assert.assertEquals(Optional.of(ContentTypes.TEXT_PLAIN_UTF8.toString()), objectMetadata.getContentType()); + } + + @Test + public void deleteBlob() throws Exception { + mockDeleteBlob(); + + //#delete-blob + final Source, NotUsed> source = + BlobService.deleteBlob(containerName() + "/" + blobName(), DeleteFile.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#delete-blob + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(0L, objectMetadata.getContentLength()); + } + + @Test + public void createDirectory() throws Exception { + mockCreateDirectory(); + + //#create-directory + final Source, NotUsed> source = FileService.createDirectory(containerName(), CreateDirectory.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#create-directory + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + final var objectMetadata = optionalObjectMetadata.get(); + Assert.assertEquals(objectMetadata.getContentLength(), 0L); + Assert.assertEquals(objectMetadata.getETag().get(), ETagRawValue()); + } + + @Test + public void deleteDirectory() throws Exception { + mockDeleteDirectory(); + + //#delete-directory + final Source, NotUsed> source = FileService.deleteDirectory(containerName(), DeleteDirectory.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#delete-directory + + final var optionalObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(optionalObjectMetadata.isPresent()); + final var objectMetadata = optionalObjectMetadata.get(); + Assert.assertEquals(objectMetadata.getContentLength(), 0L); + } + + @Test + public void createFile() throws Exception { + mockCreateFile(); + + //#create-file + final Source, NotUsed> source = + FileService.createFile(containerName() + "/" + blobName(), + CreateFile.create(contentLength(), ContentTypes.TEXT_PLAIN_UTF8)); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#create-file + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(0L, objectMetadata.getContentLength()); + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + @Ignore("Fix this test case") + @Test + public void updateRange() throws Exception { + mockCreateFile(); + + //#update-range + final Source, NotUsed> source = + FileService.updateRange(containerName() + "/" + blobName(), + UpdateFileRange.create(contentRange(), ContentTypes.TEXT_PLAIN_UTF8), + Source.single(ByteString.fromString(payload()))); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#update-range + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(0L, objectMetadata.getContentLength()); + } + + @Test + public void getFile() throws Exception { + mockGetBlob(Option.empty(), Option.empty()); + + //#get-file + final Source> source = + FileService.getFile(containerName() + "/" + blobName(), GetFile.create()); + + final CompletionStage> eventualPayload = source.runWith(Sink.seq(), system); + //#get-file + + final var actualPayload = eventualPayload.toCompletableFuture().get().stream() + .map(ByteString::utf8String).collect(Collectors.joining()); + Assert.assertEquals(actualPayload, payload()); + } + + @Test + public void getFileProperties() throws Exception { + mockGetBlobProperties(); + + //#get-file-properties + final Source, NotUsed> source = + FileService.getProperties(containerName() + "/" + blobName(), GetProperties.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#get-file-properties + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(contentLength(), objectMetadata.getContentLength()); + Assert.assertEquals(Optional.of(ContentTypes.TEXT_PLAIN_UTF8.toString()), objectMetadata.getContentType()); + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + @Ignore("Fix this test case") + @Test + public void clearRange() throws Exception { + mockClearRange(); + + //#clear-range + final Source, NotUsed> source = + FileService.clearRange(containerName() + "/" + blobName(), ClearFileRange.create(subRange())); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#clear-range + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(0L, objectMetadata.getContentLength()); + } + + @Test + public void deleteFile() throws Exception { + mockDeleteBlob(); + + //#delete-file + final Source, NotUsed> source = + FileService.deleteFile(containerName() + "/" + blobName(), DeleteFile.create()); + + final CompletionStage> optionalCompletionStage = source.runWith(Sink.head(), system); + //#delete-file + + final var maybeObjectMetadata = optionalCompletionStage.toCompletableFuture().get(); + Assert.assertTrue(maybeObjectMetadata.isPresent()); + final var objectMetadata = maybeObjectMetadata.get(); + Assert.assertEquals(Optional.of(ETagRawValue()), objectMetadata.getETag()); + Assert.assertEquals(0L, objectMetadata.getContentLength()); + } +} diff --git a/azure-storage/src/test/resources/application.conf b/azure-storage/src/test/resources/application.conf new file mode 100644 index 0000000000..1fe23177ea --- /dev/null +++ b/azure-storage/src/test/resources/application.conf @@ -0,0 +1,20 @@ +azurite { + api-version = "2024-11-04" + signing-algorithm = "HmacSHA256" + endpoint-url = "http://127.0.0.1:10000" + + credentials { + authorization-type = SharedKey + account-name = "devstoreaccount1" + account-key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==" + sas-token = "" + } +} + +akka { + http { + parsing { + max-response-reason-length = 512 + } + } +} diff --git a/azure-storage/src/test/resources/logback-test.xml b/azure-storage/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..190a393e9b --- /dev/null +++ b/azure-storage/src/test/resources/logback-test.xml @@ -0,0 +1,31 @@ + + + target/azure-blob-storage.log + false + + %d{ISO8601} %-5level [%thread] [%logger{36}] %msg%n + + + + + + %d{HH:mm:ss.SSS} %-5level [%-20.20thread] %-36.36logger{36} %msg%n%rEx + + + + + + + + + + + + + + + + + + + diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/AzuriteContainer.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/AzuriteContainer.scala new file mode 100644 index 0000000000..6f2a9df2c8 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/AzuriteContainer.scala @@ -0,0 +1,22 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import com.dimafeng.testcontainers.GenericContainer + +class AzuriteContainer + extends GenericContainer( + "mcr.microsoft.com/azure-storage/azurite:latest", + exposedPorts = Seq(10000, 10001, 10002) + ) { + + def getBlobHostAddress: String = s"http://${container.getContainerIpAddress}:${container.getMappedPort(10000)}" + + def getQueueHostAddress: String = s"http://${container.getContainerIpAddress}:${container.getMappedPort(10001)}" + + def getTableHostAddress: String = s"http://${container.getContainerIpAddress}:${container.getMappedPort(10002)}" +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageExceptionSpec.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageExceptionSpec.scala new file mode 100644 index 0000000000..01c8ce750e --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageExceptionSpec.scala @@ -0,0 +1,97 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.http.scaladsl.model.StatusCodes +import org.scalatest.flatspec.AnyFlatSpecLike +import org.scalatest.matchers.should.Matchers + +class StorageExceptionSpec extends AnyFlatSpecLike with Matchers { + + "Storage exception" should "be parsed" in { + val ex = StorageException("invalid xml", StatusCodes.NotFound) + ex.toString shouldBe """StorageException(statusCode=404 Not Found, errorCode=Content is not allowed in prolog., + | errorMessage=invalid xml, resourceName=None, resourceValue=None, reason=None) + |""".stripMargin.replaceAll(System.lineSeparator(), "") + } + + it should "parse XML containing error related to query" in { + val xml = + """ + | + | InvalidQueryParameterValue + | Value for one of the query parameters specified in the request URI is invalid. + | popreceipt + | 33537277-6a52-4a2b-b4eb-0f905051827b + | invalid receipt format + | + |""".stripMargin + + StorageException(xml, StatusCodes.BadRequest) shouldBe StorageException( + statusCode = StatusCodes.BadRequest, + errorCode = "InvalidQueryParameterValue", + errorMessage = "Value for one of the query parameters specified in the request URI is invalid.", + resourceName = Some("popreceipt"), + resourceValue = Some("33537277-6a52-4a2b-b4eb-0f905051827b"), + reason = Some("invalid receipt format") + ) + } + + it should "parse XML containing error related to header" in { + val xml = + """ + | + | InvalidHeaderValue + | Invalid header + | popreceipt + | 33537277-6a52-4a2b-b4eb-0f905051827b + | invalid receipt format + | + |""".stripMargin + + StorageException(xml, StatusCodes.BadRequest) shouldBe StorageException( + statusCode = StatusCodes.BadRequest, + errorCode = "InvalidHeaderValue", + errorMessage = "Invalid header", + resourceName = Some("popreceipt"), + resourceValue = Some("33537277-6a52-4a2b-b4eb-0f905051827b"), + reason = Some("invalid receipt format") + ) + } + + it should "parse XML containing error related to authentication" in { + val xml = + """ + | + | InvalidAuthenticationInfo + | Authentication failed + | Server failed to authenticate the request. Please refer to the information in the www-authenticate header. + | + |""".stripMargin + + StorageException(xml, StatusCodes.Unauthorized) shouldBe StorageException( + statusCode = StatusCodes.Unauthorized, + errorCode = "InvalidAuthenticationInfo", + errorMessage = "Authentication failed", + resourceName = None, + resourceValue = None, + reason = Some( + "Server failed to authenticate the request. Please refer to the information in the www-authenticate header." + ) + ) + } + + it should "survive null" in { + StorageException(null, StatusCodes.ServiceUnavailable) shouldBe + StorageException(statusCode = StatusCodes.ServiceUnavailable, + errorCode = "null", + errorMessage = "null", + resourceName = None, + resourceValue = None, + reason = None) + } +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageSettingsSpec.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageSettingsSpec.scala new file mode 100644 index 0000000000..c7207ae087 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/StorageSettingsSpec.scala @@ -0,0 +1,151 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage + +import akka.actor.ActorSystem +import akka.testkit.TestKit +import com.typesafe.config.ConfigFactory +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpecLike + +import scala.concurrent.duration._ +import scala.util.{Failure, Success, Try} + +class StorageSettingsSpec + extends TestKit(ActorSystem("StorageSettingsSystem")) + with AnyWordSpecLike + with Matchers + with BeforeAndAfterAll + with ScalaFutures { + + private def mkSettings(more: String) = + StorageSettings( + ConfigFactory + .parseString(more) + .withFallback(ConfigFactory.parseString(s""" + |api-version = "2024-11-04" + |signing-algorithm = "HmacSHA256" + |credentials { + | authorization-type = anon + | account-name = mystorageaccount + | account-key = bXlhY2NvdW50a2V5 + | sas-token = "" + |} + |retry-settings { + | max-retries = 5 + | min-backoff = 300ms + | max-backoff = 5s + | random-factor = 0.3 + |} + |""".stripMargin)) + .resolve() + ) + + override protected def afterAll(): Unit = { + super.afterAll() + + system.terminate().futureValue + } + + "StorageSettings" should { + "correctly parse config with anonymous authorization type" in { + val settings = mkSettings("") + settings.authorizationType shouldBe "anon" + } + + "correctly parse config with SharedKey authorization type" in { + val settings = mkSettings("credentials.authorization-type = SharedKey") + settings.authorizationType shouldBe "SharedKey" + } + + "correctly parse config with SharedKeyLite authorization type" in { + val settings = mkSettings("credentials.authorization-type = SharedKeyLite") + settings.authorizationType shouldBe "SharedKeyLite" + } + + "correctly parse config with sas authorization type" in { + val settings = mkSettings("credentials.authorization-type = sas") + settings.authorizationType shouldBe "sas" + } + + "fallback to anonymous authorization type when incorrectly configured" in { + val settings = mkSettings("credentials.authorization-type = dummy") + settings.authorizationType shouldBe "anon" + } + + "successfully parse account name and key" in { + val expected = AzureNameKeyCredential("mystorageaccount", "bXlhY2NvdW50a2V5") + val settings = mkSettings("") + val actual = settings.azureNameKeyCredential + actual shouldBe a[AzureNameKeyCredential] + actual.accountName shouldBe expected.accountName + actual.accountKey shouldEqual expected.accountKey + } + + "throw RuntimeException if credentials are missing" in { + Try(StorageSettings(ConfigFactory.parseString(s""" + |api-version = "2024-11-04" + |signing-algorithm = "HmacSHA256" + | + |retry-settings { + | max-retries = 3 + | min-backoff = 200ms + | max-backoff = 10s + | random-factor = 0.0 + |} + |""".stripMargin).resolve())) match { + case Failure(ex) => + ex shouldBe a[RuntimeException] + ex.getMessage shouldBe "credentials must be defined." + case Success(_) => fail("Should have thrown exception") + } + } + + "parse retry settings" in { + val settings = mkSettings("") + settings.retrySettings shouldBe RetrySettings(maxRetries = 5, + minBackoff = 300.milliseconds, + maxBackoff = 5.seconds, + randomFactor = 0.3) + } + + "use default retry settings if config is missing" in { + val settings = StorageSettings(ConfigFactory.parseString(s""" + |api-version = "2024-11-04" + |signing-algorithm = "HmacSHA256" + |credentials { + | authorization-type = anon + | account-name = mystorageaccount + | account-key = bXlhY2NvdW50a2V5 + | sas-token = "" + |} + | + |""".stripMargin).resolve()) + settings.retrySettings shouldBe RetrySettings.Default + } + + "use default retry settings if config is misconfigured" in { + val settings = mkSettings("retry-settings.min-backoff=hello") + settings.retrySettings shouldBe RetrySettings.Default + } + + "populate endpoint URL if provided" in { + val settings = mkSettings("""endpoint-url="http://localhost:1234" """) + settings.endPointUrl shouldBe defined + settings.endPointUrl.get shouldBe "http://localhost:1234" + } + + "load custom settings from a path" in { + val actual = StorageExt(system).settings("azurite") + actual.authorizationType shouldBe "SharedKey" + actual.endPointUrl shouldBe Some("http://127.0.0.1:10000") + actual.azureNameKeyCredential.accountName shouldBe "devstoreaccount1" + } + } +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/impl/auth/SignerSpec.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/impl/auth/SignerSpec.scala new file mode 100644 index 0000000000..9cf5770960 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/impl/auth/SignerSpec.scala @@ -0,0 +1,202 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package impl.auth + +import akka.actor.ActorSystem +import akka.http.scaladsl.model.{ContentTypes, HttpMethods} +import akka.http.scaladsl.model.headers.{ + `Content-Length`, + `Content-Type`, + `If-Match`, + `If-Modified-Since`, + ByteRange, + Range +} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.requests.{GetBlob, GetFile, PutBlockBlob} +import akka.testkit.TestKit +import com.azure.storage.common.StorageSharedKeyCredential +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.flatspec.AnyFlatSpecLike +import org.scalatest.matchers.should.Matchers + +import java.net.URL +import java.time.{Clock, Instant, ZoneOffset} +import scala.jdk.CollectionConverters._ + +class SignerSpec + extends TestKit(ActorSystem("SignerSystem")) + with AnyFlatSpecLike + with Matchers + with BeforeAndAfterAll + with ScalaFutures { + + private implicit val clock: Clock = Clock.fixed(Instant.parse("2024-09-02T12:00:00Z"), ZoneOffset.UTC) + private val objectPath = "my-container/MY-BLOB.csv" + private val filePath = "my-directory/my-sub-directory/my-file.csv" + private lazy val storageSettings = StorageExt(system).settings("azurite").withEndPointUrl("") + private lazy val storageSharedKeyCredential = new StorageSharedKeyCredential( + storageSettings.azureNameKeyCredential.accountName, + system.settings.config.getString("azurite.credentials.account-key") + ) + + private val leaseId = "ABC123" + private val range = ByteRange(0, 50) + private val versionId = "12345XYZ" + private val customerKey = "EqqWHbls3Y1Hp5B+IS5oUA==" + + override protected def afterAll(): Unit = { + super.afterAll() + system.terminate().futureValue + } + + it should "sign request with no parameters" in { + val requestBuilder = GetBlob() + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + + val signer = Signer(request, storageSettings) + signer.generateAuthorizationHeader shouldBe generateAuthorizationHeader(BlobType, + objectPath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0")) + } + + it should "sign request with one optional parameter" in { + val requestBuilder = GetBlob().withLeaseId(leaseId) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + val expectedValue = generateAuthorizationHeader(BlobType, + objectPath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0", LeaseIdHeaderKey -> leaseId)) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with multiple parameters" in { + val requestBuilder = GetBlob().withLeaseId(leaseId).withRange(range) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + val expectedValue = + generateAuthorizationHeader( + BlobType, + objectPath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0", + LeaseIdHeaderKey -> leaseId, + Range.name -> s"bytes=${range.first}-${range.last}") + ) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with multiple parameters and query parameter" in { + val requestBuilder = GetBlob().withLeaseId(leaseId).withRange(range).withVersionId(versionId) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + val expectedValue = + generateAuthorizationHeader( + BlobType, + objectPath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0", + LeaseIdHeaderKey -> leaseId, + Range.name -> s"bytes=${range.first}-${range.last}"), + Some(s"versionId=$versionId") + ) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with only query parameter" in { + val requestBuilder = GetFile().withVersionId(versionId) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = FileType, objectPath = filePath) + val expectedValue = + generateAuthorizationHeader( + BlobType, + filePath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0"), + Some(s"versionId=$versionId") + ) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with conditional headers" in { + val requestBuilder = + GetFile() + .addHeader("", "") + .addHeader(`If-Match`.name, "abXzWj65") + .addHeader(`If-Modified-Since`.name, getFormattedDate) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = FileType, objectPath = filePath) + val expectedValue = + generateAuthorizationHeader( + BlobType, + filePath, + HttpMethods.GET.name(), + Map(`Content-Length`.name -> "0", `If-Match`.name -> "abXzWj65", `If-Modified-Since`.name -> getFormattedDate) + ) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with required parameters" in { + val requestBuilder = PutBlockBlob(1024, ContentTypes.`text/csv(UTF-8)`) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + val expectedValue = + generateAuthorizationHeader( + BlobType, + objectPath, + HttpMethods.PUT.name(), + Map(`Content-Length`.name -> "1024", + `Content-Type`.name -> ContentTypes.`text/csv(UTF-8)`.value, + "x-ms-blob-type" -> BlockBlobType) + ) + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + it should "sign request with ServerSideEncryption enabled" in { + val requestBuilder = + PutBlockBlob(1024, ContentTypes.`text/csv(UTF-8)`) + .withServerSideEncryption(ServerSideEncryption.customerKey(customerKey)) + val request = + requestBuilder.createRequest(settings = storageSettings, storageType = BlobType, objectPath = objectPath) + val expectedValue = { + generateAuthorizationHeader( + BlobType, + objectPath, + HttpMethods.PUT.name(), + Map( + `Content-Length`.name -> "1024", + `Content-Type`.name -> ContentTypes.`text/csv(UTF-8)`.value, + "x-ms-blob-type" -> BlockBlobType, + "x-ms-encryption-algorithm" -> "AES256", + "x-ms-encryption-key" -> customerKey, + "x-ms-encryption-key-sha256" -> "Zq+2UiSyaBzexl9Y1S/TzJssWRSwsfTnPsMKA+Kew2g=" + ) + ) + } + Signer(request, storageSettings).generateAuthorizationHeader shouldBe expectedValue + } + + // generates authorization header using Azure API + private def generateAuthorizationHeader(storageType: String, + objectPath: String, + httpMethod: String, + headers: Map[String, String], + maybeQueryString: Option[String] = None) = { + val queryString = maybeQueryString.map(value => s"?$value").getOrElse("") + val allHeaders = headers ++ Map("x-ms-date" -> getFormattedDate, "x-ms-version" -> storageSettings.apiVersion) + val url = new URL( + s"https://${storageSharedKeyCredential.getAccountName}.$storageType.core.windows.net/$objectPath$queryString" + ) + storageSharedKeyCredential.generateAuthorizationHeader(url, httpMethod, allHeaders.asJava) + } + +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzureIntegrationTest.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzureIntegrationTest.scala new file mode 100644 index 0000000000..88b05bbf78 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzureIntegrationTest.scala @@ -0,0 +1,218 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.actor.ActorSystem +import akka.http.scaladsl.model.ContentTypes +import akka.http.scaladsl.model.headers.ByteRange +import akka.stream.alpakka.azure.storage.requests.{ + ClearFileRange, + CreateDirectory, + CreateFile, + DeleteDirectory, + DeleteFile, + GetFile, + GetProperties, + UpdateFileRange +} +import akka.stream.scaladsl.{Keep, Sink, Source} +import akka.util.ByteString +import org.scalatest.Ignore + +@Ignore +class AzureIntegrationTest extends StorageIntegrationSpec { + + override protected implicit val system: ActorSystem = ActorSystem("AzureIntegrationTest") + + // Azurite doesn't support FileService yet so can't add these tests in the base class + "FileService" should { + "create directory" in { + val maybeObjectMetadata = FileService + .createDirectory(directoryPath = defaultDirectoryPath, requestBuilder = CreateDirectory()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.isDefined shouldBe true + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0 + } + + "create file" in { + val maybeObjectMetadata = FileService + .createFile(objectPath = fileObjectPath, + requestBuilder = CreateFile(contentLength, ContentTypes.`text/plain(UTF-8)`)) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.isDefined shouldBe true + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0 + } + + "put range" in { + val maybeObjectMetadata = FileService + .updateRange( + objectPath = fileObjectPath, + requestBuilder = UpdateFileRange(ByteRange(0, contentLength - 1), ContentTypes.`text/plain(UTF-8)`), + payload = Source.single(ByteString(sampleText)) + ) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.isDefined shouldBe true + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0 + } + + "get file" in { + val (maybeEventualObjectMetadata, eventualText) = FileService + .getFile(objectPath = fileObjectPath, GetFile()) + .via(framing) + .map(byteString => byteString.utf8String + System.lineSeparator()) + .toMat(Sink.seq)(Keep.both) + .run() + + val objectMetadata = maybeEventualObjectMetadata.futureValue + objectMetadata.contentType shouldBe Some(ContentTypes.`text/plain(UTF-8)`.value) + objectMetadata.contentLength shouldBe sampleText.length + eventualText.futureValue.mkString("") shouldBe sampleText + } + + "get file properties" in { + val maybeObjectMetadata = + FileService + .getProperties(fileObjectPath, GetProperties()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentType shouldBe Some(ContentTypes.`text/plain(UTF-8)`.value) + objectMetadata.contentLength shouldBe sampleText.length + } + + "get file range" in { + val range = ByteRange(0, 8) + val (maybeEventualObjectMetadata, eventualText) = + FileService + .getFile(fileObjectPath, GetFile().withRange(range)) + .withAttributes(getDefaultAttributes) + .via(framing) + .map(_.utf8String) + .toMat(Sink.seq)(Keep.both) + .run() + + val objectMetadata = maybeEventualObjectMetadata.futureValue + objectMetadata.contentLength shouldBe (range.last - range.first + 1) + eventualText.futureValue.mkString("") shouldBe "The quick" + } + + "clear range" in { + val range = ByteRange(16, 24) + val maybeObjectMetadata = FileService + .clearRange(objectPath = fileObjectPath, requestBuilder = ClearFileRange(range)) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.get.contentLength shouldBe 0 + } + + "delete file" in { + val maybeObjectMetadata = + FileService + .deleteFile(fileObjectPath, DeleteFile()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.get.contentLength shouldBe 0 + } + + "get file after delete" in { + val maybeObjectMetadata = + FileService + .getProperties(fileObjectPath, GetProperties()) + .withAttributes(getDefaultAttributes) + .toMat(Sink.head)(Keep.right) + .run() + .futureValue + + maybeObjectMetadata shouldBe empty + } + + "test operations on multi level directory structure" in { + val directoryName = "sub-directory" + val directoryPath = s"$defaultDirectoryPath/$directoryName" + val filePath = s"$directoryPath/$fileName" + + // create directory + FileService + .createDirectory(directoryPath, CreateDirectory()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue shouldBe defined + + // create file + FileService + .createFile(filePath, CreateFile(contentLength, ContentTypes.`text/plain(UTF-8)`)) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue shouldBe defined + + // update content + FileService + .updateRange( + objectPath = filePath, + requestBuilder = UpdateFileRange(ByteRange(0, contentLength - 1), ContentTypes.`text/plain(UTF-8)`), + payload = Source.single(ByteString(sampleText)) + ) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue shouldBe defined + + // get file properties + FileService + .getProperties(filePath, GetProperties()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + .map(_.contentLength) shouldBe Some(sampleText.length) + + // delete file + FileService + .deleteFile(filePath, DeleteFile()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue shouldBe defined + + // delete directory + FileService + .deleteDirectory(directoryPath, DeleteDirectory()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue shouldBe defined + } + + "delete directory" in { + val maybeObjectMetadata = FileService + .deleteDirectory(directoryPath = defaultDirectoryPath, requestBuilder = DeleteDirectory()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata.isDefined shouldBe true + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0 + } + } +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzuriteIntegrationSpec.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzuriteIntegrationSpec.scala new file mode 100644 index 0000000000..2ffff35c23 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/AzuriteIntegrationSpec.scala @@ -0,0 +1,28 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.actor.ActorSystem +import akka.stream.Attributes +import com.dimafeng.testcontainers.ForAllTestContainer +import org.scalatest.Ignore + +// TODO: investigate how Azurite works, it is not even working with pure Java API +// `putBlob` operations fails with "Premature end of file." error with BadRequest. +@Ignore +class AzuriteIntegrationSpec extends StorageIntegrationSpec with ForAllTestContainer { + + override lazy val container: AzuriteContainer = new AzuriteContainer() + + override protected implicit val system: ActorSystem = ActorSystem("AzuriteIntegrationSpec") + + protected lazy val blobSettings: StorageSettings = + StorageExt(system).settings("azurite").withEndPointUrl(container.getBlobHostAddress) + + override protected def getDefaultAttributes: Attributes = StorageAttributes.settings(blobSettings) +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageIntegrationSpec.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageIntegrationSpec.scala new file mode 100644 index 0000000000..597819402d --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageIntegrationSpec.scala @@ -0,0 +1,191 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.NotUsed +import akka.actor.ActorSystem +import akka.http.scaladsl.Http +import akka.http.scaladsl.model.ContentTypes +import akka.http.scaladsl.model.headers.ByteRange +import akka.stream.Attributes +import akka.stream.alpakka.azure.storage.requests.{ + CreateContainer, + DeleteBlob, + DeleteContainer, + GetBlob, + GetProperties, + PutBlockBlob +} +import akka.stream.alpakka.testkit.scaladsl.LogCapturing +import akka.stream.scaladsl.{Flow, Framing, Keep, Sink, Source} +import akka.testkit.TestKit +import akka.util.ByteString +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.matchers.should.Matchers +import org.scalatest.{BeforeAndAfterAll, OptionValues} +import org.scalatest.wordspec.AnyWordSpecLike + +import java.security.MessageDigest +import java.util.Base64 +import scala.concurrent.duration._ +import scala.concurrent.ExecutionContext + +trait StorageIntegrationSpec + extends AnyWordSpecLike + with BeforeAndAfterAll + with Matchers + with ScalaFutures + with OptionValues + with LogCapturing { + + protected val defaultContainerName = "test-container" + // share was created manually + protected val defaultShareName = "test-share" + protected val defaultDirectoryName = "test-directory" + protected val defaultDirectoryPath = s"$defaultShareName/$defaultDirectoryName" + protected val fileName = "sample-file.txt" + protected val sampleText: String = "The quick brown fox jumps over the lazy dog." + System.lineSeparator() + protected val contentLength: Long = sampleText.length.toLong + protected val blobObjectPath = s"$defaultContainerName/$fileName" + protected val fileObjectPath = s"$defaultDirectoryPath/$fileName" + protected val framing: Flow[ByteString, ByteString, NotUsed] = + Framing.delimiter(ByteString(System.lineSeparator()), 256, allowTruncation = true) + + protected implicit val system: ActorSystem + protected implicit lazy val ec: ExecutionContext = system.dispatcher + override implicit val patienceConfig: PatienceConfig = PatienceConfig(3.minutes, 100.millis) + + override protected def afterAll(): Unit = + Http(system) + .shutdownAllConnectionPools() + .foreach(_ => TestKit.shutdownActorSystem(system)) + + protected def getDefaultAttributes: Attributes = StorageAttributes.settings(StorageSettings()) + + "BlobService" should { + "create container" in { + val maybeObjectMetadata = + BlobService + .createContainer(objectPath = defaultContainerName, requestBuilder = CreateContainer()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + } + + "put blob" in { + val maybeObjectMetadata = + BlobService + .putBlockBlob( + objectPath = blobObjectPath, + requestBuilder = PutBlockBlob(contentLength, ContentTypes.`text/plain(UTF-8)`), + payload = Source.single(ByteString(sampleText)) + ) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentMd5 shouldBe Some(calculateDigest(sampleText)) + } + + "get blob" in { + val (maybeEventualObjectMetadata, eventualText) = + BlobService + .getBlob(blobObjectPath, GetBlob()) + .withAttributes(getDefaultAttributes) + .via(framing) + .map(byteString => byteString.utf8String + System.lineSeparator()) + .toMat(Sink.seq)(Keep.both) + .run() + + val objectMetadata = maybeEventualObjectMetadata.futureValue + objectMetadata.contentMd5 shouldBe Some(calculateDigest(sampleText)) + objectMetadata.contentLength shouldBe sampleText.length + eventualText.futureValue.mkString("") shouldBe sampleText + } + + "get blob properties" in { + val maybeObjectMetadata = + BlobService + .getProperties(blobObjectPath, GetProperties()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentMd5 shouldBe Some(calculateDigest(sampleText)) + objectMetadata.contentLength shouldBe sampleText.length + } + + "get blob range" in { + val range = ByteRange.Slice(0, 8) + val (maybeEventualObjectMetadata, eventualText) = + BlobService + .getBlob(blobObjectPath, GetBlob().withRange(range)) + .withAttributes(getDefaultAttributes) + .via(framing) + .map(_.utf8String) + .toMat(Sink.seq)(Keep.both) + .run() + + val objectMetadata = maybeEventualObjectMetadata.futureValue + objectMetadata.contentLength shouldBe (range.last - range.first + 1) + eventualText.futureValue.head shouldBe "The quick" + } + + "delete blob" in { + val maybeObjectMetadata = + BlobService + .deleteBlob(blobObjectPath, DeleteBlob()) + .withAttributes(getDefaultAttributes) + .toMat(Sink.head)(Keep.right) + .run() + .futureValue + + maybeObjectMetadata.get.contentLength shouldBe 0 + } + + "get blob after delete" in { + val maybeObjectMetadata = + BlobService + .getProperties(blobObjectPath, GetProperties()) + .withAttributes(getDefaultAttributes) + .toMat(Sink.head)(Keep.right) + .run() + .futureValue + + maybeObjectMetadata shouldBe empty + } + + "delete container" in { + val maybeObjectMetadata = + BlobService + .deleteContainer(objectPath = defaultContainerName, requestBuilder = DeleteContainer()) + .withAttributes(getDefaultAttributes) + .runWith(Sink.head) + .futureValue + + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + } + } + + protected def calculateDigest(text: String): String = { + val digest = MessageDigest.getInstance("MD5") + digest.update(text.getBytes) + val bytes = digest.digest() + Base64.getEncoder.encodeToString(bytes) + } +} diff --git a/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageWireMockBase.scala b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageWireMockBase.scala new file mode 100644 index 0000000000..3da8616235 --- /dev/null +++ b/azure-storage/src/test/scala/akka/stream/alpakka/azure/storage/scaladsl/StorageWireMockBase.scala @@ -0,0 +1,315 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package akka.stream.alpakka +package azure +package storage +package scaladsl + +import akka.actor.ActorSystem +import akka.http.scaladsl.model.headers._ +import StorageWireMockBase.{config, getCallerName, initServer, AccountName, ETagValue} +import akka.http.scaladsl.model.ContentTypes +import akka.testkit.TestKit +import com.github.tomakehurst.wiremock.WireMockServer +import com.github.tomakehurst.wiremock.client.WireMock +import com.github.tomakehurst.wiremock.client.WireMock._ +import com.github.tomakehurst.wiremock.core.WireMockConfiguration.wireMockConfig +import com.github.tomakehurst.wiremock.extension.requestfilter.{ + RequestFilterAction, + RequestWrapper, + StubRequestFilterV2 +} +import com.github.tomakehurst.wiremock.http.Request +import com.github.tomakehurst.wiremock.stubbing.{ServeEvent, StubMapping} +import com.typesafe.config.{Config, ConfigFactory} + +import scala.util.Try + +abstract class StorageWireMockBase(_system: ActorSystem, val _wireMockServer: WireMockServer) extends TestKit(_system) { + + private val port = _wireMockServer.port() + protected val mock = new WireMock("localhost", port) + + // test data + protected val containerName = "my-container" + protected val blobName = "my-blob.txt" + protected val payload = "The quick brown fox jumps over the lazy dog." + protected val contentLength: Long = payload.length.toLong + protected val contentRange: ByteRange.Slice = ByteRange(0, contentLength - 1) + protected val subRange: ByteRange.Slice = ByteRange(4, 8) + + private def this(mock: WireMockServer) = + this( + ActorSystem(getCallerName(StorageWireMockBase.getClass), config(mock.port()).withFallback(ConfigFactory.load())), + mock + ) + + def this() = { + this(initServer()) + system.registerOnTermination(stopWireMockServer()) + } + + protected def mockCreateContainer(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName?restype=container")) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockDeleteContainer(): StubMapping = + mock.register( + delete(urlEqualTo(s"/$AccountName/$containerName?restype=container")) + .willReturn( + aResponse() + .withStatus(202) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockCreateDirectory(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName?restype=directory")) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockDeleteDirectory(): StubMapping = + mock.register( + delete(urlEqualTo(s"/$AccountName/$containerName?restype=directory")) + .willReturn( + aResponse() + .withStatus(202) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockPutBlockBlob(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader(BlobTypeHeaderKey, equalTo(BlockBlobType)) + .withRequestBody(equalTo(payload)) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, payload.length.toString) + .withHeader(`Content-Type`.name, "text/plain; charset=UTF-8") + ) + ) + + protected def mockPutPageBlob(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader(BlobTypeHeaderKey, equalTo(PageBlobType)) + .withHeader(PageBlobContentLengthHeaderKey, equalTo("512")) + .withHeader(PageBlobSequenceNumberHeaderKey, equalTo("0")) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + .withHeader(`Content-Type`.name, "text/plain; charset=UTF-8") + ) + ) + + protected def mockPutAppendBlob(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader(BlobTypeHeaderKey, equalTo(AppendBlobType)) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + .withHeader(`Content-Type`.name, "text/plain; charset=UTF-8") + ) + ) + + protected def mockGetBlob(versionId: Option[String] = None, leaseId: Option[String] = None): StubMapping = + mock.register( + get(urlPathEqualTo(s"/$AccountName/$containerName/$blobName")) + .withQueryParam("versionId", toStringValuePattern(versionId)) + .withHeader(LeaseIdHeaderKey, toStringValuePattern(leaseId)) + .willReturn( + aResponse() + .withStatus(200) + .withHeader(ETag.name, ETagValue) + .withBody(payload) + ) + ) + + protected def mockGetBlobWithServerSideEncryption(): StubMapping = + mock.register( + get(urlPathEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader("x-ms-encryption-algorithm", equalTo("AES256")) + .withHeader("x-ms-encryption-key", equalTo("SGVsbG9Xb3JsZA==")) + .withHeader("x-ms-encryption-key-sha256", equalTo("hy5OUM6ZkNiwQTMMR8nd0Rvsa1A66ThqmdqFhOm7EsQ=")) + .willReturn( + aResponse() + .withStatus(200) + .withHeader(ETag.name, ETagValue) + .withBody(payload) + ) + ) + + protected def mockGetBlobWithRange(): StubMapping = + mock.register( + get(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader(Range.name, equalTo(s"bytes=${subRange.first}-${subRange.last}")) + .willReturn( + aResponse() + .withStatus(200) + .withHeader(ETag.name, ETagValue) + .withBody(payload.substring(subRange.first.toInt, subRange.last.toInt + 1)) + ) + ) + + protected def mockGetBlobProperties(): StubMapping = + mock.register( + head(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .willReturn( + aResponse() + .withStatus(200) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, payload.length.toString) + .withHeader(`Content-Type`.name, "text/plain; charset=UTF-8") + ) + ) + + protected def mockDeleteBlob(): StubMapping = + mock.register( + delete(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .willReturn( + aResponse() + .withStatus(202) + .withHeader(ETag.name, ETagValue) + ) + ) + + protected def mockCreateFile(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName")) + .withHeader(XMsContentLengthHeaderKey, equalTo(contentLength.toString)) + .withHeader(FileTypeHeaderKey, equalTo("file")) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockUpdateRange(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName?comp=range")) + .withHeader(Range.name, equalTo(s"bytes=0-${contentLength - 1}")) + .withHeader(FileWriteTypeHeaderKey, equalTo("update")) + .withRequestBody(equalTo(payload)) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mockClearRange(): StubMapping = + mock.register( + put(urlEqualTo(s"/$AccountName/$containerName/$blobName?comp=range")) + .withHeader(`Content-Type`.name, equalTo(ContentTypes.NoContentType.toString())) + .withHeader(Range.name, equalTo(s"bytes=${subRange.first}-${subRange.last}")) + .withHeader(FileWriteTypeHeaderKey, equalTo("clear")) + .willReturn( + aResponse() + .withStatus(201) + .withHeader(ETag.name, ETagValue) + .withHeader(`Content-Length`.name, "0") + ) + ) + + protected def mock404s(): StubMapping = + mock.register( + any(anyUrl()) + .willReturn(aResponse().withStatus(404).withBody(""" + | + | ResourceNotFound + | The specified resource doesn't exist. + | + |""".stripMargin)) + ) + + private def toStringValuePattern(maybeValue: Option[String]) = maybeValue.map(equalTo).getOrElse(absent()) + + private def stopWireMockServer(): Unit = _wireMockServer.stop() +} + +object StorageWireMockBase { + + val AccountName = "teststoreaccount" + val ETagRawValue = "fba9dede5f27731c9771645a39863328" + val ETagValue = s""""$ETagRawValue"""" + + def initServer(): WireMockServer = { + val server = new WireMockServer( + wireMockConfig() + .extensions(new RemoveDuplicateContentLengthHeader()) + .dynamicPort() + ) + server.start() + server + } + + def getCallerName(clazz: Class[_]): String = { + val s = (Thread.currentThread.getStackTrace map (_.getClassName) drop 1) + .dropWhile(_ matches "(java.lang.Thread|.*WireMockBase.?$)") + val reduced = s.lastIndexWhere(_ == clazz.getName) match { + case -1 => s + case z => s drop (z + 1) + } + reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_") + } + + def config(proxyPort: Int): Config = ConfigFactory.parseString(s""" + |akka.http.client.log-unencrypted-network-bytes = 1000 + |akka.http.parsing.max-to-strict-bytes=infinite + |${StorageSettings.ConfigPath} { + | endpoint-url = "http://localhost:$proxyPort" + | credentials { + | authorization-type = anon + | account-name = $AccountName + | } + |} + |""".stripMargin) + + private class RemoveDuplicateContentLengthHeader extends StubRequestFilterV2 { + override def filter(request: Request, serveEvent: ServeEvent): RequestFilterAction = { + val headerName = `Content-Length`.name + + val updatedRequest = + Try(request.getHeader(headerName)).toOption match { + case Some(contentLengthValue) => + RequestWrapper + .create() + .removeHeader(headerName) + .addHeader(headerName, contentLengthValue) + .wrap(request) + + case None => request + } + + RequestFilterAction.continueWith(updatedRequest) + } + + override def getName: String = "remove-duplicate-content-length-headers" + } +} diff --git a/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala b/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala new file mode 100644 index 0000000000..5083b0ce80 --- /dev/null +++ b/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala @@ -0,0 +1,70 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package docs.scaladsl + +import akka.http.scaladsl.model.ContentTypes +import akka.http.scaladsl.model.headers.{ByteRange, RawHeader} +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.requests.{CreateFile, GetBlob, PutBlockBlob} +import akka.stream.alpakka.testkit.scaladsl.LogCapturing +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + +class RequestBuilderSpec extends AnyFlatSpec with Matchers with LogCapturing { + + it should "create request builder with default values" in { + + //#simple-request-builder + val requestBuilder = GetBlob() + //#simple-request-builder + + requestBuilder.versionId shouldBe empty + requestBuilder.range shouldBe empty + requestBuilder.leaseId shouldBe empty + requestBuilder.sse shouldBe empty + requestBuilder.additionalHeaders shouldBe empty + } + + it should "create request builder with values" in { + + //#populate-request-builder + val requestBuilder = GetBlob().withLeaseId("my-lease-id").withRange(ByteRange(0, 25)) + //#populate-request-builder + + requestBuilder.leaseId shouldBe Some("my-lease-id") + requestBuilder.range shouldBe Some(ByteRange(0, 25)) + requestBuilder.sse shouldBe empty + } + + it should "create request builder with initial values" in { + + //#request-builder-with-initial-values + val requestBuilder = CreateFile(256L, ContentTypes.`text/plain(UTF-8)`) + //#request-builder-with-initial-values + + requestBuilder.leaseId shouldBe empty + requestBuilder.maxFileSize shouldBe 256L + requestBuilder.contentType shouldBe ContentTypes.`text/plain(UTF-8)` + } + + it should "populate request builder with ServerSideEncryption" in { + + //#request-builder-with-sse + val requestBuilder = PutBlockBlob(256L, ContentTypes.`text/plain(UTF-8)`) + .withServerSideEncryption(ServerSideEncryption.customerKey("SGVsbG9Xb3JsZA==")) + //#request-builder-with-sse + + requestBuilder.sse shouldBe Some(ServerSideEncryption.customerKey("SGVsbG9Xb3JsZA==")) + } + + it should "populate request builder with additional headers" in { + + //#request-builder-with-additional-headers + val requestBuilder = GetBlob().addHeader("If-Match", "foobar") + //#request-builder-with-additional-headers + + requestBuilder.additionalHeaders shouldBe Seq(RawHeader("If-Match", "foobar")) + } +} diff --git a/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala b/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala new file mode 100644 index 0000000000..93a0a792c8 --- /dev/null +++ b/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala @@ -0,0 +1,484 @@ +/* + * Copyright (C) since 2016 Lightbend Inc. + */ + +package docs.scaladsl + +import akka.NotUsed +import akka.http.scaladsl.model.{ContentTypes, StatusCodes} +import akka.stream.alpakka.azure.storage.StorageException +import akka.stream.alpakka.azure.storage.headers.ServerSideEncryption +import akka.stream.alpakka.azure.storage.scaladsl.StorageWireMockBase +import akka.stream.alpakka.azure.storage.scaladsl.StorageWireMockBase.ETagRawValue +import akka.stream.alpakka.testkit.scaladsl.LogCapturing +import akka.stream.scaladsl.{Keep, Sink, Source} +import akka.util.ByteString +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpecLike +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, OptionValues} + +import scala.concurrent.Future +import scala.concurrent.duration._ + +class StorageSpec + extends StorageWireMockBase + with AnyWordSpecLike + with BeforeAndAfterAll + with BeforeAndAfterEach + with Matchers + with ScalaFutures + with OptionValues + with LogCapturing { + + override implicit val patienceConfig: PatienceConfig = PatienceConfig(3.minutes, 100.millis) + + override protected def afterEach(): Unit = mock.removeMappings() + + "AzureStorage Blob connector" should { + + "create container" in { + mockCreateContainer() + + //#create-container + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.CreateContainer + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.createContainer(containerName, CreateContainer()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#create-container + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + objectMetadata.eTag shouldBe Some(ETagRawValue) + } + + "delete container" in { + mockDeleteContainer() + + //#delete-container + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.DeleteContainer + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.deleteContainer(containerName, DeleteContainer()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#delete-container + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + "put block blob" ignore { + mockPutBlockBlob() + + //#put-block-blob + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.PutBlockBlob + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.putBlockBlob( + objectPath = s"$containerName/$blobName", + payload = Source.single(ByteString(payload)), + requestBuilder = PutBlockBlob(contentLength, ContentTypes.`text/plain(UTF-8)`) + ) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#put-block-blob + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe contentLength + objectMetadata.eTag shouldBe Some(ETagRawValue) + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + "put page blob" ignore { + mockPutPageBlob() + + //#put-page-blob + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.PutPageBlock + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.putPageBlock( + objectPath = s"$containerName/$blobName", + requestBuilder = PutPageBlock(512L, ContentTypes.`text/plain(UTF-8)`).withBlobSequenceNumber(0) + ) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#put-page-blob + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + objectMetadata.eTag shouldBe Some(ETagRawValue) + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + "put append blob" ignore { + mockPutAppendBlob() + + //#put-append-blob + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.PutAppendBlock + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.putAppendBlock(objectPath = s"$containerName/$blobName", + requestBuilder = PutAppendBlock(ContentTypes.`text/plain(UTF-8)`)) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#put-append-blob + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0L + objectMetadata.eTag shouldBe Some(ETagRawValue) + } + + "get blob" in { + mockGetBlob() + + //#get-blob + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", GetBlob()) + + val eventualText = source.toMat(Sink.seq)(Keep.right).run() + //#get-blob + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe payload + } + + "get blob with versionId" in { + mockGetBlob(Some("versionId")) + + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", GetBlob().withVersionId("versionId")) + + val eventualText = source.toMat(Sink.seq)(Keep.right).run() + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe payload + } + + "get blob with optional header" in { + mockGetBlob(leaseId = Some("leaseId")) + + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", GetBlob().withLeaseId("leaseId")) + + val eventualText = source.toMat(Sink.seq)(Keep.right).run() + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe payload + } + + "get blob with ServerSideEncryption" in { + mockGetBlobWithServerSideEncryption() + + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", + GetBlob().withServerSideEncryption(ServerSideEncryption.customerKey("SGVsbG9Xb3JsZA=="))) + + val eventualText = source.toMat(Sink.seq)(Keep.right).run() + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe payload + } + + "get blob from non-existing container" in { + mock404s() + + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", GetBlob()) + + val eventualMetadata = source.toMat(Sink.seq)(Keep.right).run() + eventualMetadata.failed.futureValue shouldBe + StorageException( + statusCode = StatusCodes.NotFound, + errorCode = "ResourceNotFound", + errorMessage = "The specified resource doesn't exist.", + resourceName = None, + resourceValue = None, + reason = None + ) + } + + "get blob range" in { + mockGetBlobWithRange() + + //#get-blob-range + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetBlob + + val source: Source[ByteString, Future[ObjectMetadata]] = + BlobService.getBlob(objectPath = s"$containerName/$blobName", requestBuilder = GetBlob().withRange(subRange)) + + val eventualText: Future[Seq[ByteString]] = source.toMat(Sink.seq)(Keep.right).run() + //#get-blob-range + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe "quick" + } + + "get blob properties" in { + mockGetBlobProperties(); + + //#get-blob-properties + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetProperties + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.getProperties(objectPath = s"$containerName/$blobName", GetProperties()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#get-blob-properties + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe contentLength + metadata.contentType shouldBe Some(ContentTypes.`text/plain(UTF-8)`.value) + } + + "delete blob" in { + mockDeleteBlob() + + //#delete-blob + import akka.stream.alpakka.azure.storage.scaladsl.BlobService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.DeleteBlob + + val source: Source[Option[ObjectMetadata], NotUsed] = + BlobService.deleteBlob(objectPath = s"$containerName/$blobName", DeleteBlob()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#delete-blob + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe 0L + } + } + + "AzureStorage File connector" should { + + "create directory" in { + mockCreateDirectory() + + //#create-directory + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.CreateDirectory + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.createDirectory(directoryPath = containerName, requestBuilder = CreateDirectory()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#create-directory + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe 0L + } + + "create file" in { + mockCreateFile() + + //#create-file + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.CreateFile + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.createFile(objectPath = s"$containerName/$blobName", + requestBuilder = CreateFile(contentLength, ContentTypes.`text/plain(UTF-8)`)) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#create-file + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe 0L + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + "update range" ignore { + mockUpdateRange() + + //#update-range + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.UpdateFileRange + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.updateRange( + objectPath = s"$containerName/$blobName", + payload = Source.single(ByteString(payload)), + requestBuilder = UpdateFileRange(contentRange, ContentTypes.`text/plain(UTF-8)`) + ) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.toMat(Sink.head)(Keep.right).run() + //#update-range + + val maybeObjectMetadata = eventualMaybeMetadata.futureValue + maybeObjectMetadata shouldBe defined + val objectMetadata = maybeObjectMetadata.get + objectMetadata.contentLength shouldBe 0 + objectMetadata.eTag shouldBe Some(ETagRawValue) + } + + "get file" in { + mockGetBlob() + + //#get-file + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetFile + + val source: Source[ByteString, Future[ObjectMetadata]] = + FileService.getFile(objectPath = s"$containerName/$blobName", GetFile()) + + val eventualText: Future[Seq[ByteString]] = source.toMat(Sink.seq)(Keep.right).run() + //#get-file + + eventualText.futureValue.map(_.utf8String).mkString("") shouldBe payload + } + + "get file properties" in { + mockGetBlobProperties() + + //#get-file-properties + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.GetProperties + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.getProperties(objectPath = s"$containerName/$blobName", GetProperties()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.toMat(Sink.head)(Keep.right).run() + //#get-file-properties + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe contentLength + metadata.contentType shouldBe Some(ContentTypes.`text/plain(UTF-8)`.value) + } + + // TODO: There are couple of issues, firstly there are two `Content-Length` headers being added, one by `putBlob` + // function and secondly by, most likely, by WireMock. Need to to figure out how to tell WireMock not to add `Content-Length` + // header, secondly once that resolve then we get `akka.http.scaladsl.model.EntityStreamException`. + "clear range" ignore { + mockClearRange() + + //#clear-range + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.ClearFileRange + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.clearRange(objectPath = s"$containerName/$blobName", requestBuilder = ClearFileRange(subRange)) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#clear-range + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe 0L + } + + "delete file" in { + mockDeleteBlob() + + //#delete-file + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.DeleteFile + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.deleteFile(objectPath = s"$containerName/$blobName", DeleteFile()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#delete-file + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.eTag shouldBe Some(ETagRawValue) + metadata.contentLength shouldBe 0L + } + + "delete directory" in { + mockDeleteDirectory() + + //#delete-directory + import akka.stream.alpakka.azure.storage.scaladsl.FileService + import akka.stream.alpakka.azure.storage.ObjectMetadata + import akka.stream.alpakka.azure.storage.requests.DeleteDirectory + + val source: Source[Option[ObjectMetadata], NotUsed] = + FileService.deleteDirectory(directoryPath = containerName, requestBuilder = DeleteDirectory()) + + val eventualMaybeMetadata: Future[Option[ObjectMetadata]] = source.runWith(Sink.head) + //#delete-directory + + val maybeMetadata = eventualMaybeMetadata.futureValue + maybeMetadata shouldBe defined + val metadata = maybeMetadata.get + metadata.contentLength shouldBe 0L + } + } +} diff --git a/build.sbt b/build.sbt index f0fc260dc7..c5fed5ee9c 100644 --- a/build.sbt +++ b/build.sbt @@ -10,6 +10,7 @@ lazy val alpakka = project avroparquetTests, awslambda, azureStorageQueue, + azureStorage, cassandra, couchbase, csv, @@ -138,6 +139,19 @@ lazy val azureStorageQueue = alpakkaProject( Scala3.settings ) +lazy val azureStorage = alpakkaProject( + "azure-storage", + "azure.storage", + Dependencies.AzureStorage, + Scala3.settings, + Test / fork := true, + Test / envVars := Map( + "AZURE_STORAGE_AUTHORIZATION_TYPE" -> "SharedKey", + "AZURE_STORAGE_ACCOUNT_NAME" -> "", + "AZURE_STORAGE_ACCOUNT_KEY" -> "" + ) +).settings(mimaPreviousArtifacts := Set.empty) // FIXME remove after first release + lazy val cassandra = alpakkaProject("cassandra", "cassandra", Dependencies.Cassandra) .settings(Scala3.settings) diff --git a/docs/src/main/paradox/azure-storage.md b/docs/src/main/paradox/azure-storage.md new file mode 100644 index 0000000000..edb81cfbcf --- /dev/null +++ b/docs/src/main/paradox/azure-storage.md @@ -0,0 +1,264 @@ +# Azure Storage + +Azure Storage connector provides Akka Stream Source for Azure Storage. Currently only supports `Blob` and `File` services. For detail about these services please read [Azure docs](https://learn.microsoft.com/en-us/rest/api/storageservices/). + +@@project-info{ projectId="azure-storage" } + +## Artifacts + +The Akka dependencies are available from Akka's library repository. To access them there, you need to configure the URL for this repository. + +@@repository [sbt,Maven,Gradle] { +id="akka-repository" +name="Akka library repository" +url="https://repo.akka.io/maven" +} + +Additionally, add the dependencies as below. + +@@dependency [sbt,Maven,Gradle] { +group=com.lightbend.akka +artifact=akka-stream-alpakka-azure-storage_$scala.binary.version$ +version=$project.version$ +symbol2=AkkaVersion +value2=$akka.version$ +group2=com.typesafe.akka +artifact2=akka-stream_$scala.binary.version$ +version2=AkkaVersion +symbol3=AkkaHttpVersion +value3=$akka-http.version$ +group3=com.typesafe.akka +artifact3=akka-http_$scala.binary.version$ +version3=AkkaHttpVersion +group4=com.typesafe.akka +artifact4=akka-http-xml_$scala.binary.version$ +version4=AkkaHttpVersion +} + +The table below shows direct dependencies of this module and the second tab shows all libraries it depends on transitively. + +@@dependencies { projectId="azure-storage" } + +## Configuration + +The settings for the Azure Storage connector are read by default from `alpakka.azure-storage` configuration section. Credentials are defined in `credentials` section of [`reference.conf`](/azure-storage/src/main/resources/reference.conf). + +Scala +: @@snip [snip](/azure-storage/src/main/resources/reference.conf) { #azure-credentials } + +Java +: @@snip [snip](/azure-storage/src/main/resources/reference.conf) { #azure-credentials } + +At minimum following configurations needs to be set: + +* `authorization-type`, this is the type of authorization to use as described [here](https://learn.microsoft.com/en-us/rest/api/storageservices/authorize-requests-to-azure-storage), possible values are `anon`, `SharedKey`, or `sas`. Environment variable `AZURE_STORAGE_AUTHORIZATION_TYPE` can be set to override this configuration. +* `account-name`, this is the name of the blob storage or file share. Environment variable `AZURE_STORAGE_ACCOUNT_NAME` can be set to override this configuration. +* `account-key`, Account key to use to create authorization signature, mandatory for `SharedKey` or `SharedKeyLite` authorization types, as described [here](https://learn.microsoft.com/en-us/rest/api/storageservices/authorize-with-shared-key). Environment variable `AZURE_STORAGE_ACCOUNT_KEY` can be set to override this configuration. +* `sas-token` if authorization type is `sas`. Environment variable `AZURE_STORAGE_SAS_TOKEN` can be set to override this configuration. + +## Building request + +Each function takes two parameters `objectPath` and `requestBuilder`. The `objectPath` is a `/` separated string of the path of the blob +or file, for example, `my-container/my-blob` or `my-share/my-directory/my-file`. + +Each request builder is subclass of [`RequestBuilder`](/azure-storage/src/main/scala/akka/stream/alpakka/azure/storage/requests/RequestBuilder.scala) which knows how to construct request for the given operation. + +### Create simple request builder with default values + +In this example `GetBlob` builder is initialized without any optional field. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala) { #simple-request-builder } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java) { #simple-request-builder } + +### Create request builder initialized with optional fields + +In this example `GetBlob` builder is initialized with given `leaseId` and `range` fields. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala) { #populate-request-builder } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java) { #populate-request-builder } + +### Create request builder initialized with required fields + +In this example `CreateFile` builder is initialized with `maxFileSize` and `contentType` fields, which are required fields for `CreateFile` operation. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala) { #request-builder-with-initial-values } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java) { #request-builder-with-initial-values } + +### Create request builder with ServerSideEncryption + +`ServerSideEncryption` can be initialized in similar fashion. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala) { #request-builder-with-sse } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java) { #request-builder-with-sse } + +### Create request builder with additional headers + +Some operations allow you to add additional headers, for `GetBlob` you can specify `If-Match` header, which specify this header to perform the operation only if the resource's ETag matches the value specified, this can be done by calling `addHeader` function. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/RequestBuilderSpec.scala) { #request-builder-with-additional-headers } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/RequestBuilderTest.java) { #request-builder-with-additional-headers } + +## Supported operations on Blob service + +### Create Container + +The [`Create Container`](https://learn.microsoft.com/en-us/rest/api/storageservices/create-container) operation creates a new container under the specified account. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #create-container } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #create-container } + +### Delete Container + +The [`Delete Container`](https://learn.microsoft.com/en-us/rest/api/storageservices/delete-container) operation creates existing container under the specified account. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #delete-container } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #delete-container } + +### Put Block Blob + +The [`Put Block Blob`](https://learn.microsoft.com/en-us/rest/api/storageservices/put-blob) operation creates a new block or updates the content of an existing block blob. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #put-block-blob } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #put-block-blob } + +### Get Blob + +The [`Get Blob`](https://learn.microsoft.com/en-us/rest/api/storageservices/get-blob) operation reads or downloads a blob from the system, including its metadata and properties. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #get-blob } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #get-blob } + +In order to download a range of a file's data you can use overloaded method which additionally takes `ByteRange` as argument. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #get-blob-range } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #get-blob-range } + +### Get blob properties without downloading blob + +The [`Get Blob Properties`](https://learn.microsoft.com/en-us/rest/api/storageservices/get-blob-properties) operation returns all user-defined metadata, standard HTTP properties, and system properties for the blob. (**Note:** Current implementation does not return user-defined metadata.) + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #get-blob-properties } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #get-blob-properties } + +### Delete Blob + +The [`Delete Blob`](https://learn.microsoft.com/en-us/rest/api/storageservices/delete-blob) operation deletes the specified blob. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #delete-blob } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #delete-blob } + +## Supported operations on File service + +### Create File + +The [`Create File`](https://learn.microsoft.com/en-us/rest/api/storageservices/create-file) operation creates a new file or replaces a file. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #create-file } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #create-file } + +### Update Range + +The [`Update Range`](https://learn.microsoft.com/en-us/rest/api/storageservices/put-range) operation writes a range of bytes to a file. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #update-range } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #update-range } + +Range can be cleared using `ClearRange` function. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #clear-range } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #clear-range } + +### Create Directory + +The [`Create Directory`](https://learn.microsoft.com/en-us/rest/api/storageservices/create-directory) operation creates a new container under the specified account. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #create-directory } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #create-directory } + +### Delete Directory + +The [`Delete Directory`](https://learn.microsoft.com/en-us/rest/api/storageservices/delete-directory) operation creates existing container under the specified account. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #delete-directory } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #delete-directory } + +### Get File + +The [`Get File`](https://learn.microsoft.com/en-us/rest/api/storageservices/get-file) operation reads or downloads a file from the system, including its metadata and properties. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #get-file } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #get-file } + +### Get file properties without downloading file + +The [`Get File Properties`](https://learn.microsoft.com/en-us/rest/api/storageservices/get-file-properties) operation returns all user-defined metadata, standard HTTP properties, and system properties for the file. (**Note:** Current implementation does not return user-defined metatdata.) + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #get-file-properties } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #get-file-properties } + +### Delete Blob + +The [`Delete File`](https://learn.microsoft.com/en-us/rest/api/storageservices/delete-file2) operation immediately removes the file from the storage account. + +Scala +: @@snip [snip](/azure-storage/src/test/scala/docs/scaladsl/StorageSpec.scala) { #delete-file } + +Java +: @@snip [snip](/azure-storage/src/test/java/docs/javadsl/StorageTest.java) { #delete-file } diff --git a/docs/src/main/paradox/index.md b/docs/src/main/paradox/index.md index c0ab8a5558..0ff4614233 100644 --- a/docs/src/main/paradox/index.md +++ b/docs/src/main/paradox/index.md @@ -23,6 +23,7 @@ The [Alpakka project](https://doc.akka.io/libraries/alpakka/current/) is an init * [AWS S3](s3.md) * [AWS SNS](sns.md) * [AWS SQS](sqs.md) +* [Azure Storage](azure-storage.md) * [Azure Storage Queue](azure-storage-queue.md) * [Couchbase](couchbase.md) * [Elasticsearch](elasticsearch.md) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 2efdfa9d9d..e29c46053a 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -115,6 +115,16 @@ object Dependencies { ) ) + val AzureStorage = Seq( + libraryDependencies ++= Seq( + "com.typesafe.akka" %% "akka-http" % AkkaHttpVersion, + "com.typesafe.akka" %% "akka-http-xml" % AkkaHttpVersion, + // for testing authorization signature + "com.azure" % "azure-storage-common" % "12.26.1" % Test, + wiremock + ) + ) + val CassandraVersionInDocs = "4.0" val CassandraDriverVersion = "4.17.0" val CassandraDriverVersionInDocs = "4.17" diff --git a/project/project-info.conf b/project/project-info.conf index abf953785a..82d322ed3f 100644 --- a/project/project-info.conf +++ b/project/project-info.conf @@ -96,6 +96,24 @@ project-info { } ] } + azure-storage: ${project-info.shared-info} { + title: "Alpakka Azure Storage" + jpms-name: "akka.stream.alpakka.azure.storage" + issues.url: ${project-info.labels}"azure-storage" + levels: [ + { + readiness: CommunityDriven + since: "2024-08-28" + since-version: "8.0.0" + } + ] + api-docs: [ + { + url: ${project-info.scaladoc}"azure/storage/index.html" + text: "API (Scaladoc)" + } + ] + } azure-storage-queue: ${project-info.shared-info} { title: "Alpakka Azure Storage Queue" jpms-name: "akka.stream.alpakka.azure.storagequeue"