Skip to content

Commit

Permalink
Add ETag-caching (and AWS SDK v2) support
Browse files Browse the repository at this point in the history
This change adds these improvements:

* Facia data is only re-downloaded & re-parsed if the S3 content has _changed_ (thanks to [ETag-caching](https://github.com/guardian/etag-caching))
* Independence from AWS SDK version (v1 vs v2) _(this PR can replace #286

The [ETag-caching](https://github.com/guardian/etag-caching) library itself is also being used in DotCom PROD, introduced with guardian/frontend#26338.

### Usage

```scala
import com.gu.etagcaching.aws.sdkv2.s3.S3ObjectFetching
import com.gu.facia.client.{ApiClient, Environment}
import software.amazon.awssdk.services.s3.S3AsyncClient

val s3AsyncClient = S3AsyncClient.builder().region(...).credentialsProvider(...).build()

val apiClient = ApiClient.withCaching(
  "facia-tool-store",
  Environment.Prod,
  S3ObjectFetching.byteArraysWith(s3AsyncClient)
)
```

_PR using this updated version of the FAPI client: https://github.com/guardian/ophan/pull/6741_

### Independence from AWS SDK version (v1 vs v2)

Ideally, the whole of `facia-scala-client` would be independent of AWS SDK version - we'd _like_ consumers of this library to be able to use whatever AWS SDK version they want, without us pulling in dependency on either SDK version.

For `facia-scala-client` this is an attainable goal, as the only AWS API action it performs is fetching from S3, and [guardian/etag-caching](https://github.com/guardian/etag-caching) provides the [`S3ByteArrayFetching`](https://github.com/guardian/etag-caching/blob/v6.0.0/aws-s3/base/src/main/scala/com/gu/etagcaching/aws/s3/package.scala#L6-L22) abstraction that encapsulates this action without tying to a specific AWS SDK version.

Due to legacy code compatibility, we can't completely remove AWS SDK v1 from `fapi-client` for now, but we _have_ removed it from `fapi-client-core`, which is the new home of `com.gu.facia.client.ApiClient`, which is now a trait, with 2 constructor methods that provide different implementations:

* **`ApiClient()`**  - legacy, using the existing `com.gu.facia.client.S3Client` abstraction on S3 behaviour
* **`ApiClient.withCaching()`** - provides ETag-based caching and is independent of AWS SDK version - the consumer just needs to provide an appropriate instance of `com.gu.etagcaching.aws.sdkv2.s3.S3ObjectFetching` (ie with `"com.gu.etag-caching" %% "aws-s3-sdk-v2"` and `S3ObjectFetching.byteArraysWith(s3AsyncClient)`, introduced with guardian/etag-caching#65)

### Solved problems

* **Noisy logging associated with absent collection JSON** - the `etag-caching` library has been updated with guardian/etag-caching#56 to avoid excessive logging that would occur in the Facia client, due to it typically trying to access collections that aren't yet persisted: #32.
  • Loading branch information
rtyley committed Jan 8, 2025
1 parent 48f5559 commit 2175db0
Show file tree
Hide file tree
Showing 14 changed files with 228 additions and 77 deletions.
15 changes: 12 additions & 3 deletions build.sbt
Original file line number Diff line number Diff line change
Expand Up @@ -6,10 +6,10 @@ name := "facia-api-client"

description := "Scala client for The Guardian's Facia JSON API"

ThisBuild / scalaVersion := "2.13.14"
ThisBuild / scalaVersion := "2.13.15"

val sonatypeReleaseSettings = Seq(
releaseVersion := fromAggregatedAssessedCompatibilityWithLatestRelease().value,
// releaseVersion := fromAggregatedAssessedCompatibilityWithLatestRelease().value,
releaseCrossBuild := true, // true if you cross-build the project for multiple Scala versions
releaseProcess := Seq[ReleaseStep](
checkSnapshotDependencies,
Expand Down Expand Up @@ -37,7 +37,14 @@ def artifactProducingSettings(supportScala3: Boolean) = Seq(
libraryDependencies += scalaTest
)

// fapi-client-core is independent of AWS SDK version.
lazy val fapiClient_core = (project in file("fapi-client-core")).settings(
libraryDependencies += eTagCachingS3Base,
artifactProducingSettings(supportScala3 = true)
)

lazy val root = (project in file(".")).aggregate(
fapiClient_core,
faciaJson_play28,
faciaJson_play29,
faciaJson_play30,
Expand All @@ -55,9 +62,10 @@ def playJsonSpecificProject(module: String, playJsonVersion: PlayJsonVersion) =
)

def faciaJson(playJsonVersion: PlayJsonVersion) = playJsonSpecificProject("facia-json", playJsonVersion)
.dependsOn(fapiClient_core)
.settings(
libraryDependencies ++= Seq(
awsSdk,
awsS3SdkV1, // ideally, this would be pushed out to a separate FAPI artifact, or just not used directly at all
commonsIo,
playJsonVersion.lib,
"org.scala-lang.modules" %% "scala-collection-compat" % "2.11.0",
Expand All @@ -69,6 +77,7 @@ def faciaJson(playJsonVersion: PlayJsonVersion) = playJsonSpecificProject("facia
def fapiClient(playJsonVersion: PlayJsonVersion) = playJsonSpecificProject("fapi-client", playJsonVersion)
.settings(
libraryDependencies ++= Seq(
eTagCachingS3SupportForTesting,
contentApi,
contentApiDefault,
commercialShared,
Expand Down
Original file line number Diff line number Diff line change
@@ -1,19 +1,19 @@
package com.gu.facia.client

import com.amazonaws.regions.{Region, Regions}
import com.amazonaws.services.s3.{AmazonS3, AmazonS3ClientBuilder}
import com.amazonaws.regions.Regions
import com.amazonaws.services.s3.model.AmazonS3Exception
import com.amazonaws.services.s3.{AmazonS3, AmazonS3ClientBuilder}
import org.apache.commons.io.IOUtils

import scala.concurrent.{ExecutionContext, Future, blocking}
import scala.util.Try

/** For mocking in tests, but also to allow someone to define a properly asynchronous S3 client. (The one in the AWS
* SDK is unfortunately synchronous only.)
*/
trait S3Client {
def get(bucket: String, path: String): Future[FaciaResult]
}

/**
* Legacy wrapper around the AWS SDK v1, which is itself a legacy version of the AWS SDK.
*
* Ideally, don't use this class. It is passed to the legacy `com.gu.facia.client.ApiClient()` constructor -
* instead use AWS SDK v2 and the `ApiClient.withCaching()` constructor.
*/
case class AmazonSdkS3Client(client: AmazonS3)(implicit executionContext: ExecutionContext) extends S3Client {
def get(bucket: String, path: String): Future[FaciaResult] = Future {
blocking {
Expand Down
91 changes: 68 additions & 23 deletions facia-json/src/main/scala/com/gu/facia/client/ApiClient.scala
Original file line number Diff line number Diff line change
@@ -1,37 +1,82 @@
package com.gu.facia.client

import com.gu.facia.client.models.{ConfigJson, CollectionJson}
import com.gu.etagcaching.FreshnessPolicy.AlwaysWaitForRefreshedValue
import com.gu.etagcaching.aws.s3.{ObjectId, S3ByteArrayFetching}
import com.gu.etagcaching.fetching.Fetching
import com.gu.etagcaching.{ConfigCache, ETagCache}
import com.gu.facia.client.models.{CollectionJson, ConfigJson}
import play.api.libs.json.{Format, Json}

import scala.concurrent.{ExecutionContext, Future}

object ApiClient {
val Encoding = "utf-8"
trait ApiClient {
def config: Future[ConfigJson]

def collection(id: String): Future[Option[CollectionJson]]
}

case class ApiClient(
object ApiClient {
private val Encoding = "utf-8"

/**
* Legacy constructor for creating a client that does not support caching. Use `ApiClient.withCaching()` instead.
*/
def apply(
bucket: String,
/** e.g., CODE, PROD, DEV ... */
environment: String,
environment: String, // e.g., CODE, PROD, DEV ...
s3Client: S3Client
)(implicit executionContext: ExecutionContext) {
import com.gu.facia.client.ApiClient._

private def retrieve[A: Format](key: String): Future[Option[A]] = s3Client.get(bucket, key) map {
case FaciaSuccess(bytes) =>
Some(Json.fromJson[A](Json.parse(new String(bytes, Encoding))) getOrElse {
throw new JsonDeserialisationError(s"Could not deserialize JSON in $bucket, $key")
})
case FaciaNotAuthorized(message) => throw new BackendError(message)
case FaciaNotFound(_) => None
case FaciaUnknownError(message) => throw new BackendError(message)
)(implicit executionContext: ExecutionContext): ApiClient = new ApiClient {
val env: Environment = Environment(environment)

private def retrieve[A: Format](key: String): Future[Option[A]] = s3Client.get(bucket, key).map(translateFaciaResult[A](_))

def config: Future[ConfigJson] =
retrieve[ConfigJson](env.configS3Path).map(getOrWarnAboutMissingConfig)

def collection(id: String): Future[Option[CollectionJson]] =
retrieve[CollectionJson](env.collectionS3Path(id))
}

private def getOrWarnAboutMissingConfig(configOpt: Option[ConfigJson]): ConfigJson =
configOpt.getOrElse(throw BackendError("Config was missing!! NOT GOOD!"))

private def translateFaciaResult[B: Format](faciaResult: FaciaResult): Option[B] = faciaResult match {
case FaciaSuccess(bytes) => Some(parseBytes(bytes))
case FaciaNotAuthorized(message) => throw BackendError(message)
case FaciaNotFound(_) => None
case FaciaUnknownError(message) => throw BackendError(message)
}

def config: Future[ConfigJson] =
retrieve[ConfigJson](s"$environment/frontsapi/config/config.json").map(_ getOrElse {
throw new BackendError("Config was missing!! OH MY GOD")
})
private def parseBytes[B: Format](bytes: Array[Byte]): B =
Json.fromJson[B](Json.parse(new String(bytes, Encoding))) getOrElse {
throw JsonDeserialisationError(s"Could not deserialize JSON")
}

/**
* @param s3Fetching see scaladoc on `S3ByteArrayFetching` i.e. use `S3ObjectFetching.byteArraysWith(s3AsyncClient)`
*/
def withCaching(
bucket: String,
environment: Environment,
s3Fetching: S3ByteArrayFetching,
configureCollectionCache: ConfigCache = _.maximumSize(10000) // at most 1GB RAM worst case
)(implicit ec: ExecutionContext): ApiClient = new ApiClient {
private val fetching =
s3Fetching.keyOn[String](path => ObjectId(bucket, path))

def eTagCache[B: Format](configureCache: ConfigCache) = new ETagCache(
fetching.thenParsing(parseBytes[B]),
AlwaysWaitForRefreshedValue,
configureCache
)

private val configCache = eTagCache[ConfigJson](_.maximumSize(1))
private val collectionCache = eTagCache[CollectionJson](configureCollectionCache)

def collection(id: String): Future[Option[CollectionJson]] =
retrieve[CollectionJson](s"$environment/frontsapi/collection/$id/collection.json")
override def config: Future[ConfigJson] =
configCache.get(environment.configS3Path).map(getOrWarnAboutMissingConfig)

override def collection(id: String): Future[Option[CollectionJson]] =
collectionCache.get(environment.collectionS3Path(id))
}
}
53 changes: 42 additions & 11 deletions facia-json/src/test/scala/com/gu/facia/client/ApiClientSpec.scala
Original file line number Diff line number Diff line change
@@ -1,33 +1,64 @@
package com.gu.facia.client

import com.gu.etagcaching.aws.s3.{ObjectId, S3ByteArrayFetching}
import com.gu.etagcaching.fetching.{ETaggedData, Fetching, Missing, MissingOrETagged}
import com.gu.facia.client.lib.ResourcesHelper
import org.scalatest.OptionValues
import org.scalatest.concurrent.{IntegrationPatience, ScalaFutures}
import org.scalatest.flatspec.AnyFlatSpec
import org.scalatest.matchers.should.Matchers

import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.Future
import scala.concurrent.{ExecutionContext, Future}
import scala.util.hashing.MurmurHash3

/**
* This is used only for testing, it's a dummy implementation of S3ByteArrayFetching that
* just loads blobs from the `src/test/resources` folder, rather than hitting S3.
*/
object FakeS3Fetching extends S3ByteArrayFetching with ResourcesHelper {
private def pretendETagFor(bytes: Array[Byte]): String = MurmurHash3.bytesHash(bytes).toHexString

override def fetch(objectId: ObjectId)(implicit ec: ExecutionContext): Future[MissingOrETagged[Array[Byte]]] = Future {
slurpBytes(objectId.key).fold(Missing: MissingOrETagged[Array[Byte]]) { bytes =>
ETaggedData(pretendETagFor(bytes), bytes)
}
}

override def fetchOnlyIfETagChanged(objectId: ObjectId, oldETag: String)(implicit ec: ExecutionContext): Future[Option[MissingOrETagged[Array[Byte]]]] = {
fetch(objectId).map {
case taggedData: ETaggedData[_] =>
Option.unless(oldETag == taggedData.eTag)(taggedData) // simulate a Not-Modified response, if there's no change in ETag
case x => Some(x)
}
}
}

class ApiClientSpec extends AnyFlatSpec with Matchers with OptionValues with ScalaFutures with IntegrationPatience {
import scala.concurrent.ExecutionContext.Implicits.global

object FakeS3Client extends S3Client with ResourcesHelper {
override def get(bucket: String, path: String): Future[FaciaResult] = Future {
slurpOrDie(path)
}
}

val client: ApiClient = ApiClient("not used", "DEV", FakeS3Client)
val bucketName = "not used"
val legacyClient: ApiClient = ApiClient(bucketName, "DEV", FakeS3Client)
val cachingClient: ApiClient = ApiClient.withCaching(bucketName, Environment.Dev, FakeS3Fetching)

"ApiClient" should "fetch the config" in {
val config = client.config.futureValue
// Here we're testing that both the legacy client & the new caching client satisfy the same test criteria
for ((name, client) <- Map("legacy" -> legacyClient, "caching" -> cachingClient)) {
s"$name ApiClient" should "fetch the config" in {
val config = client.config.futureValue

config.collections should have size 334
config.fronts should have size 79
}
config.collections should have size 334
config.fronts should have size 79
}

it should "fetch a collection" in {
val collectionOpt = client.collection("2409-31b3-83df0-de5a").futureValue
it should "fetch a collection" in {
val collectionOpt = cachingClient.collection("2409-31b3-83df0-de5a").futureValue

collectionOpt.value.live should have size 8
collectionOpt.value.live should have size 8
}
}
}
Original file line number Diff line number Diff line change
@@ -1,27 +1,21 @@
package com.gu.facia.client.lib

import com.amazonaws.auth.{ AWSStaticCredentialsProvider, BasicAWSCredentials}
import com.amazonaws.regions.Regions
import com.amazonaws.services.s3.{AmazonS3, AmazonS3ClientBuilder}
import com.gu.facia.client.models.{CollectionJson, ConfigJson}
import com.gu.facia.client.{AmazonSdkS3Client, ApiClient}
import com.gu.facia.client.{ApiClient, Environment}
import play.api.libs.json.{Format, Json}

import scala.concurrent.ExecutionContext.Implicits.global
import scala.concurrent.Future

object Amazon {
val amazonS3Client = AmazonS3ClientBuilder.standard().withRegion(Regions.EU_WEST_1).withCredentials(new AWSStaticCredentialsProvider(new BasicAWSCredentials("key", "pass"))).build()
}

class ApiTestClient extends ApiClient("bucket", "DEV", AmazonSdkS3Client(Amazon.amazonS3Client)) with ResourcesHelper {
class ApiTestClient extends ApiClient with ResourcesHelper {
private val environment = Environment.Dev

private def retrieve[A: Format](key: String): Future[Option[A]] =
Future.successful(slurp(key).map(Json.parse).flatMap(_.asOpt[A]))

override def config: Future[ConfigJson] =
retrieve[ConfigJson](s"$environment/frontsapi/config/config.json").map(_.get)
retrieve[ConfigJson](environment.configS3Path).map(_.get)

override def collection(id: String): Future[Option[CollectionJson]] =
retrieve[CollectionJson](s"$environment/frontsapi/collection/$id/collection.json")
retrieve[CollectionJson](environment.collectionS3Path(id))
}
Original file line number Diff line number Diff line change
@@ -1,11 +1,17 @@
package com.gu.facia.client.lib

import com.gu.facia.client.FaciaSuccess
import org.apache.commons.io.IOUtils

import java.nio.charset.StandardCharsets.UTF_8

trait ResourcesHelper {
def slurpBytes(path: String): Option[Array[Byte]] =
Option(getClass.getClassLoader.getResource(path)).map(url => IOUtils.toByteArray(url.openStream()))

def slurp(path: String): Option[String] =
Option(getClass.getClassLoader.getResource(path)).map(scala.io.Source.fromURL(_).mkString)
slurpBytes(path).map(bytes => new String(bytes, UTF_8))

def slurpOrDie(path: String) = slurp(path).map(_.getBytes).map(FaciaSuccess.apply).getOrElse {
def slurpOrDie(path: String) = slurpBytes(path).map(FaciaSuccess.apply).getOrElse {
throw new RuntimeException(s"Required resource $path not on class path") }
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
package com.gu.facia.client

case class Environment(name: String) {
private val s3PathPrefix: String = s"$name/frontsapi"

val configS3Path: String = s"$s3PathPrefix/config/config.json"
def collectionS3Path(id: String): String = s"$s3PathPrefix/collection/$id/collection.json"
}

object Environment {
val Prod = Environment("PROD")
val Code = Environment("CODE")
val Dev = Environment("DEV")
val Test = Environment("TEST")
}
16 changes: 16 additions & 0 deletions fapi-client-core/src/main/scala/com/gu/facia/client/S3Client.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
package com.gu.facia.client

import scala.concurrent.Future

/**
* Legacy class for mocking in tests, but also previously used to allow library users to define a properly
* asynchronous S3 client (back when the one in the AWS SDK was synchronous only).
*
* Note that use of `S3Client` is now discouraged, as `facia-scala-client` now supports caching using the
* `etag-caching` library, which provides its own more powerful abstraction for fetching & parsing data, and
* `com.gu.etagcaching.aws.sdkv2.s3.S3ObjectFetching`, a ready-made implementation of fetching that includes
* support for ETags & Conditional Requests, decoding S3 exceptions, etc.
*/
trait S3Client {
def get(bucket: String, path: String): Future[FaciaResult]
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
package com.gu.facia.client

import org.scalatest.Assertion
import org.scalatest.flatspec.AnyFlatSpec

import java.lang.Class.forName

/**
* Ideally, the whole of `facia-scala-client` would be independent of AWS SDK version - we'd _like_ consumers
* of this library to be able to use whatever AWS SDK version they want, without us pulling in dependency on
* either SDK version. For `facia-scala-client`, this is an attainable goal, as the only AWS API action it performs
* is fetching from S3, and https://github.com/guardian/etag-caching provides the `S3ByteArrayFetching` abstraction
* that encapsulates this action without tying to a specific AWS SDK version.
*
* Due to legacy code compatibility, we can't completely remove AWS SDK v1 from `fapi-client` for now, but
* we _have_ removed it from `fapi-client-core`. This `SdkIndependenceTest` keeps us honest, and ensures we
* don't accidentally re-add the dependency.
*/
class SdkIndependenceTest extends AnyFlatSpec {

private def assertClassIsUnavailable(className: String): Assertion =
assertThrows[ClassNotFoundException](forName(className))

"fapi-client-core" should "be independent of AWS SDKs (ie should not rely on AWS SDK v1 or v2)" in {
assertClassIsUnavailable("com.amazonaws.services.s3.AmazonS3") // AWS SDK v1
assertClassIsUnavailable("software.amazon.awssdk.auth.credentials.AwsCredentialsProvider") // AWS SDK v2
}
it should "be independent of Play-Json, or at least that would be nice" in {
assertClassIsUnavailable("play.api.libs.json.Format")
}
}
Loading

0 comments on commit 2175db0

Please sign in to comment.