From 1742fd11b0cae168b0dd249b07b7c0063851fc62 Mon Sep 17 00:00:00 2001 From: Peter Vlugter <59895+pvlugter@users.noreply.github.com> Date: Fri, 25 Oct 2024 19:04:31 +1300 Subject: [PATCH] feat: DynamoDB projections (#1222) --- .../workflows/integration-tests-dynamodb.yml | 54 + .../docs/javadsl/ProjectionDocExample.java | 470 +++++ .../src/test/resources/logback-test.xml | 26 + .../akka/projection/TestStatusObserver.scala | 79 + .../DynamoDBOffsetStoreStateSpec.scala | 113 ++ ...ynamoDBTimestampOffsetProjectionSpec.scala | 1587 +++++++++++++++++ .../DynamoDBTimestampOffsetStoreSpec.scala | 1061 +++++++++++ .../dynamodb/EventSourcedPubSubSpec.scala | 236 +++ .../akka/projection/dynamodb/TestActors.scala | 116 ++ .../akka/projection/dynamodb/TestClock.scala | 52 + .../akka/projection/dynamodb/TestConfig.scala | 29 + .../akka/projection/dynamodb/TestData.scala | 44 + .../projection/dynamodb/TestDbLifecycle.scala | 85 + .../TestSourceProviderWithInput.scala | 99 + ...ojectionTimeToLiveSettingsDocExample.scala | 58 + .../docs/scaladsl/ProjectionDocExample.scala | 467 +++++ .../scripts/create-tables.sh | 14 + .../src/main/resources/reference.conf | 64 + .../dynamodb/DynamoDBProjectionSettings.scala | 190 ++ .../internal/DynamoDBHandlerAdapter.scala | 50 + .../internal/DynamoDBOffsetStore.scala | 733 ++++++++ .../internal/DynamoDBProjectionImpl.scala | 704 ++++++++ .../dynamodb/internal/OffsetPidSeqNr.scala | 23 + .../dynamodb/internal/OffsetStoreDao.scala | 440 +++++ .../dynamodb/javadsl/CreateTables.scala | 35 + .../dynamodb/javadsl/DynamoDBProjection.scala | 168 ++ .../javadsl/DynamoDBTransactHandler.scala | 65 + .../dynamodb/scaladsl/CreateTables.scala | 108 ++ .../scaladsl/DynamoDBProjection.scala | 267 +++ .../scaladsl/DynamoDBTransactHandler.scala | 48 + build.sbt | 28 +- docker-files/docker-compose-dynamodb.yml | 8 + docs/src/main/paradox/dynamodb.md | 346 ++++ docs/src/main/paradox/index.md | 1 + project/Dependencies.scala | 31 +- project/project-info.conf | 11 + 36 files changed, 7903 insertions(+), 7 deletions(-) create mode 100644 .github/workflows/integration-tests-dynamodb.yml create mode 100644 akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java create mode 100644 akka-projection-dynamodb-integration/src/test/resources/logback-test.xml create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/TestStatusObserver.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBOffsetStoreStateSpec.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetProjectionSpec.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetStoreSpec.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/EventSourcedPubSubSpec.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestActors.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestClock.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestConfig.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestData.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestDbLifecycle.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestSourceProviderWithInput.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/projection/docs/config/ProjectionTimeToLiveSettingsDocExample.scala create mode 100644 akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala create mode 100644 akka-projection-dynamodb/scripts/create-tables.sh create mode 100644 akka-projection-dynamodb/src/main/resources/reference.conf create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/DynamoDBProjectionSettings.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBHandlerAdapter.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBOffsetStore.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBProjectionImpl.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetPidSeqNr.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetStoreDao.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/CreateTables.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBProjection.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBTransactHandler.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/CreateTables.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBProjection.scala create mode 100644 akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBTransactHandler.scala create mode 100644 docker-files/docker-compose-dynamodb.yml create mode 100644 docs/src/main/paradox/dynamodb.md diff --git a/.github/workflows/integration-tests-dynamodb.yml b/.github/workflows/integration-tests-dynamodb.yml new file mode 100644 index 000000000..51853c551 --- /dev/null +++ b/.github/workflows/integration-tests-dynamodb.yml @@ -0,0 +1,54 @@ +name: Integration Tests for DynamoDB + +on: + pull_request: + push: + branches: + - main + tags-ignore: [ v.* ] + +jobs: + test-dynamodb: + name: Run DynamoDB tests + runs-on: ubuntu-22.04 + if: github.repository == 'akka/akka-projection' + strategy: + fail-fast: false + matrix: + include: + # - { jdkVersion: "1.11.0", jvmName: "temurin:1.11.0", extraOpts: '-J-XX:+UnlockExperimentalVMOptions -J-XX:+UseJVMCICompiler' } + - { jdkVersion: "1.17.0", jvmName: "temurin:1.17.0", extraOpts: '' } + + steps: + - name: Checkout + # https://github.com/actions/checkout/releases + # v4.1.1 + uses: actions/checkout@b4ffde65f46336ab88eb53be808477a3936bae11 + with: + fetch-depth: 0 + + - name: Checkout GitHub merge + if: github.event.pull_request + run: |- + git fetch origin pull/${{ github.event.pull_request.number }}/merge:scratch + git checkout scratch + + - name: Cache Coursier cache + # https://github.com/coursier/cache-action/releases + # v6.4.5 + uses: coursier/cache-action@1ff273bff02a8787bc9f1877d347948af647956d + + - name: Set up JDK ${{ matrix.jdkVersion }} + # https://github.com/coursier/setup-action/releases + # v1.3.0 + uses: coursier/setup-action@70323223454ac2a9eb2de46f389b4d045cbcdea5 + with: + jvm: ${{ matrix.jvmName }} + + - name: Start DynamoDB + run: |- + docker compose -f docker-files/docker-compose-dynamodb.yml up --wait + + - name: Run integration tests with with Scala and Java ${{ matrix.jdkVersion }} + run: |- + sbt akka-projection-dynamodb-integration/test -Dakka.warn-on-no-license-key=false diff --git a/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java b/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java new file mode 100644 index 000000000..2269638d7 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java @@ -0,0 +1,470 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package projection.docs.javadsl; + +import java.time.Duration; +import java.time.Instant; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionStage; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import akka.Done; +import akka.actor.typed.ActorSystem; +import akka.cluster.sharding.typed.javadsl.EntityTypeKey; +import akka.serialization.jackson.CborSerializable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +// #create-tables +import akka.persistence.dynamodb.util.ClientProvider; +import akka.projection.dynamodb.DynamoDBProjectionSettings; +import akka.projection.dynamodb.javadsl.CreateTables; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; + +// #create-tables + +// #handler +// #grouped-handler +import akka.projection.javadsl.Handler; +// #grouped-handler +// #handler + +// #transact-handler +// #grouped-transact-handler +import akka.projection.dynamodb.javadsl.DynamoDBTransactHandler; +// #grouped-transact-handler +// #transact-handler + +// #handler +// #transact-handler +// #grouped-handler +// #grouped-transact-handler +import software.amazon.awssdk.services.dynamodb.model.*; + +// #grouped-transact-handler +// #grouped-handler +// #transact-handler +// #handler + +// #init-projections +import akka.cluster.sharding.typed.ShardedDaemonProcessSettings; +import akka.cluster.sharding.typed.javadsl.ShardedDaemonProcess; +import akka.japi.Pair; +import akka.persistence.dynamodb.query.javadsl.DynamoDBReadJournal; +import akka.persistence.query.Offset; +import akka.persistence.query.typed.EventEnvelope; +import akka.projection.Projection; +import akka.projection.ProjectionBehavior; +// #projection-imports +import akka.projection.ProjectionId; +import akka.projection.dynamodb.javadsl.DynamoDBProjection; +// #projection-imports +import akka.projection.eventsourced.javadsl.EventSourcedProvider; +import akka.projection.javadsl.SourceProvider; + +import java.util.Optional; + +// #projection-imports + +// #projection-imports +// #init-projections + +public class ProjectionDocExample { + + public static void createTables(ActorSystem system) throws Exception { + // #create-tables + String dynamoDBConfigPath = "akka.projection.dynamodb"; + + DynamoDBProjectionSettings settings = + DynamoDBProjectionSettings.create(system.settings().config().getConfig(dynamoDBConfigPath)); + + DynamoDbAsyncClient client = ClientProvider.get(system).clientFor(settings.useClient()); + + // create journal table, synchronously + CreateTables.createTimestampOffsetStoreTable(system, settings, client, /*deleteIfExists:*/ true) + .toCompletableFuture() + .get(10, TimeUnit.SECONDS); + // #create-tables + } + + public static class ShoppingCart { + public static EntityTypeKey ENTITY_TYPE_KEY = + EntityTypeKey.create(Command.class, "ShoppingCart"); + + public interface Command extends CborSerializable {} + + public interface Event { + String getCartId(); + } + + public static class CheckedOut implements Event { + + public final String cartId; + public final Instant eventTime; + + public CheckedOut(String cartId, Instant eventTime) { + this.cartId = cartId; + this.eventTime = eventTime; + } + + public String getCartId() { + return cartId; + } + + @Override + public String toString() { + return "CheckedOut(" + cartId + "," + eventTime + ")"; + } + } + } + + // #handler + public class ShoppingCartHandler extends Handler> { + + private final DynamoDbAsyncClient client; + + private final Logger logger = LoggerFactory.getLogger(getClass()); + + public ShoppingCartHandler(DynamoDbAsyncClient client) { + this.client = client; + } + + @Override + public CompletionStage process(EventEnvelope envelope) { + ShoppingCart.Event event = envelope.event(); + if (event instanceof ShoppingCart.CheckedOut) { + ShoppingCart.CheckedOut checkedOut = (ShoppingCart.CheckedOut) event; + + logger.info( + "Shopping cart {} was checked out at {}", checkedOut.cartId, checkedOut.eventTime); + + Map attributes = + Map.of( + "id", AttributeValue.fromS(checkedOut.cartId), + "time", AttributeValue.fromN(String.valueOf(checkedOut.eventTime.toEpochMilli()))); + + CompletableFuture response = + client.putItem(PutItemRequest.builder().tableName("orders").item(attributes).build()); + + return response.thenApply(__ -> Done.getInstance()); + + } else { + logger.debug("Shopping cart {} changed by {}", event.getCartId(), event); + return CompletableFuture.completedFuture(Done.getInstance()); + } + } + } + + // #handler + + // #transact-handler + public class ShoppingCartTransactHandler + implements DynamoDBTransactHandler> { + + private final Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public CompletionStage> process( + EventEnvelope envelope) { + ShoppingCart.Event event = envelope.event(); + if (event instanceof ShoppingCart.CheckedOut) { + ShoppingCart.CheckedOut checkedOut = (ShoppingCart.CheckedOut) event; + + logger.info( + "Shopping cart {} was checked out at {}", checkedOut.cartId, checkedOut.eventTime); + + Map attributes = + Map.of( + "id", AttributeValue.fromS(checkedOut.cartId), + "time", AttributeValue.fromN(String.valueOf(checkedOut.eventTime.toEpochMilli()))); + + List items = + List.of( + TransactWriteItem.builder() + .put(Put.builder().tableName("orders").item(attributes).build()) + .build()); + + return CompletableFuture.completedFuture(items); + + } else { + logger.debug("Shopping cart {} changed by {}", event.getCartId(), event); + return CompletableFuture.completedFuture(Collections.emptyList()); + } + } + } + + // #transact-handler + + // #grouped-handler + public class GroupedShoppingCartHandler extends Handler>> { + + private final DynamoDbAsyncClient client; + + private final Logger logger = LoggerFactory.getLogger(getClass()); + + public GroupedShoppingCartHandler(DynamoDbAsyncClient client) { + this.client = client; + } + + @Override + public CompletionStage process(List> envelopes) { + List items = + envelopes.stream() + .flatMap( + envelope -> { + ShoppingCart.Event event = envelope.event(); + + if (event instanceof ShoppingCart.CheckedOut) { + ShoppingCart.CheckedOut checkedOut = (ShoppingCart.CheckedOut) event; + + logger.info( + "Shopping cart {} was checked out at {}", + checkedOut.cartId, + checkedOut.eventTime); + + Map attributes = + Map.of( + "id", + AttributeValue.fromS(checkedOut.cartId), + "time", + AttributeValue.fromN( + String.valueOf(checkedOut.eventTime.toEpochMilli()))); + + return Stream.of( + WriteRequest.builder() + .putRequest(PutRequest.builder().item(attributes).build()) + .build()); + + } else { + logger.debug("Shopping cart {} changed by {}", event.getCartId(), event); + return Stream.empty(); + } + }) + .collect(Collectors.toList()); + + CompletableFuture response = + client.batchWriteItem( + BatchWriteItemRequest.builder().requestItems(Map.of("orders", items)).build()); + + return response.thenApply(__ -> Done.getInstance()); + } + } + + // #grouped-handler + + // #grouped-transact-handler + public class GroupedShoppingCartTransactHandler + implements DynamoDBTransactHandler>> { + + private final Logger logger = LoggerFactory.getLogger(getClass()); + + @Override + public CompletionStage> process( + List> envelopes) { + List items = + envelopes.stream() + .flatMap( + envelope -> { + ShoppingCart.Event event = envelope.event(); + if (event instanceof ShoppingCart.CheckedOut) { + ShoppingCart.CheckedOut checkedOut = (ShoppingCart.CheckedOut) event; + + logger.info( + "Shopping cart {} was checked out at {}", + checkedOut.cartId, + checkedOut.eventTime); + + Map attributes = + Map.of( + "id", AttributeValue.fromS(checkedOut.cartId), + "time", + AttributeValue.fromN( + String.valueOf(checkedOut.eventTime.toEpochMilli()))); + + return Stream.of( + TransactWriteItem.builder() + .put(Put.builder().tableName("orders").item(attributes).build()) + .build()); + + } else { + logger.debug("Shopping cart {} changed by {}", event.getCartId(), event); + return Stream.empty(); + } + }) + .collect(Collectors.toList()); + + return CompletableFuture.completedFuture(items); + } + } + + // #grouped-transact-handler + + public class InitExample { + private final ActorSystem system; + + public InitExample(ActorSystem system) { + this.system = system; + } + + // #init-projections + void initProjections() { + ShardedDaemonProcess.get(system) + .initWithContext( + ProjectionBehavior.Command.class, + "ShoppingCartProjection", + 4, + daemonContext -> { + List> sliceRanges = + EventSourcedProvider.sliceRanges( + system, DynamoDBReadJournal.Identifier(), daemonContext.totalProcesses()); + Pair sliceRange = sliceRanges.get(daemonContext.processNumber()); + return ProjectionBehavior.create(createProjection(sliceRange)); + }, + ShardedDaemonProcessSettings.create(system), + Optional.of(ProjectionBehavior.stopMessage())); + } + + Projection> createProjection( + Pair sliceRange) { + int minSlice = sliceRange.first(); + int maxSlice = sliceRange.second(); + + String entityType = ShoppingCart.ENTITY_TYPE_KEY.name(); + + SourceProvider> sourceProvider = + EventSourcedProvider.eventsBySlices( + system, DynamoDBReadJournal.Identifier(), entityType, minSlice, maxSlice); + + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + Optional settings = Optional.empty(); + + return DynamoDBProjection.exactlyOnce( + projectionId, settings, sourceProvider, ShoppingCartTransactHandler::new, system); + } + // #init-projections + } + + public void exactlyOnceExample( + SourceProvider> sourceProvider, + int minSlice, + int maxSlice, + ActorSystem system) { + // #exactly-once + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + + Optional settings = Optional.empty(); + + Projection> projection = + DynamoDBProjection.exactlyOnce( + projectionId, settings, sourceProvider, ShoppingCartTransactHandler::new, system); + // #exactly-once + } + + public void atLeastOnceExample( + SourceProvider> sourceProvider, + int minSlice, + int maxSlice, + DynamoDbAsyncClient client, + ActorSystem system) { + // #at-least-once + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + + Optional settings = Optional.empty(); + + int saveOffsetAfterEnvelopes = 100; + Duration saveOffsetAfterDuration = Duration.ofMillis(500); + + Projection> projection = + DynamoDBProjection.atLeastOnce( + projectionId, + settings, + sourceProvider, + () -> new ShoppingCartHandler(client), + system) + .withSaveOffset(saveOffsetAfterEnvelopes, saveOffsetAfterDuration); + // #at-least-once + } + + public void exactlyOnceGroupedWithinExample( + SourceProvider> sourceProvider, + int minSlice, + int maxSlice, + ActorSystem system) { + // #exactly-once-grouped-within + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + + Optional settings = Optional.empty(); + + int groupAfterEnvelopes = 20; + Duration groupAfterDuration = Duration.ofMillis(500); + + Projection> projection = + DynamoDBProjection.exactlyOnceGroupedWithin( + projectionId, + settings, + sourceProvider, + GroupedShoppingCartTransactHandler::new, + system) + .withGroup(groupAfterEnvelopes, groupAfterDuration); + // #exactly-once-grouped-within + } + + public void atLeastOnceGroupedWithinExample( + SourceProvider> sourceProvider, + int minSlice, + int maxSlice, + DynamoDbAsyncClient client, + ActorSystem system) { + // #at-least-once-grouped-within + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + + Optional settings = Optional.empty(); + + int groupAfterEnvelopes = 20; + Duration groupAfterDuration = Duration.ofMillis(500); + + Projection> projection = + DynamoDBProjection.atLeastOnceGroupedWithin( + projectionId, + settings, + sourceProvider, + () -> new GroupedShoppingCartHandler(client), + system) + .withGroup(groupAfterEnvelopes, groupAfterDuration); + // #at-least-once-grouped-within + } + + public void projectionWithSecondPlugin( + SourceProvider> sourceProvider, + int minSlice, + int maxSlice, + DynamoDbAsyncClient client, + ActorSystem system) { + // #projection-settings + ProjectionId projectionId = + ProjectionId.of("ShoppingCarts", "carts-" + minSlice + "-" + maxSlice); + + Optional settings = + Optional.of( + DynamoDBProjectionSettings.create( + system.settings().config().getConfig("second-projection-dynamodb"))); + + Projection> projection = + DynamoDBProjection.atLeastOnce( + projectionId, settings, sourceProvider, () -> new ShoppingCartHandler(client), system); + // #projection-settings + } +} diff --git a/akka-projection-dynamodb-integration/src/test/resources/logback-test.xml b/akka-projection-dynamodb-integration/src/test/resources/logback-test.xml new file mode 100644 index 000000000..e1aa88c4d --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/resources/logback-test.xml @@ -0,0 +1,26 @@ + + + + + + [%date{ISO8601}] [%level] [%logger] [%X{akkaAddress}] [%marker] [%thread] - %msg%n + + + + + + + + + + + + + + + + + + + + diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/TestStatusObserver.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/TestStatusObserver.scala new file mode 100644 index 000000000..42a9d89cc --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/TestStatusObserver.scala @@ -0,0 +1,79 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection + +import akka.actor.typed.ActorRef + +// FIXME this is an exact copy of akka-projection-core-test/src/test/scala/akka/projection/TestStatusObserver.scala + +object TestStatusObserver { + sealed trait Status + + case object Started extends Status + case object Failed extends Status + case object Stopped extends Status + + final case class OffsetProgress[Envelope](envelope: Envelope) extends Status + + trait EnvelopeProgress[Envelope] extends Status + final case class Before[Envelope](envelope: Envelope) extends EnvelopeProgress[Envelope] + final case class After[Envelope](envelope: Envelope) extends EnvelopeProgress[Envelope] + + final case class Err[Envelope](env: Envelope, cause: Throwable) extends Status { + // don't include cause message in equals + override def equals(obj: Any): Boolean = obj match { + case Err(`env`, e) => e.getClass == cause.getClass + case _ => false + } + + override def hashCode(): Int = env.hashCode() + } +} + +class TestStatusObserver[Envelope]( + probe: ActorRef[TestStatusObserver.Status], + lifecycle: Boolean = false, + offsetProgressProbe: Option[ActorRef[TestStatusObserver.OffsetProgress[Envelope]]] = None, + beforeEnvelopeProbe: Option[ActorRef[TestStatusObserver.Before[Envelope]]] = None, + afterEnvelopeProbe: Option[ActorRef[TestStatusObserver.After[Envelope]]] = None) + extends StatusObserver[Envelope] { + import TestStatusObserver._ + + override def started(projectionId: ProjectionId): Unit = { + if (lifecycle) + probe ! Started + } + + override def failed(projectionId: ProjectionId, cause: Throwable): Unit = { + if (lifecycle) + probe ! Failed + } + + override def stopped(projectionId: ProjectionId): Unit = { + if (lifecycle) + probe ! Stopped + } + + override def beforeProcess(projectionId: ProjectionId, envelope: Envelope): Unit = { + beforeEnvelopeProbe.foreach(_ ! Before(envelope)) + } + + override def afterProcess(projectionId: ProjectionId, envelope: Envelope): Unit = { + afterEnvelopeProbe.foreach(_ ! After(envelope)) + } + + override def offsetProgress(projectionId: ProjectionId, envelope: Envelope): Unit = { + offsetProgressProbe.foreach(_ ! OffsetProgress(envelope)) + } + + override def error( + projectionId: ProjectionId, + envelope: Envelope, + cause: Throwable, + recoveryStrategy: HandlerRecoveryStrategy): Unit = { + probe ! Err(envelope, cause) + } + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBOffsetStoreStateSpec.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBOffsetStoreStateSpec.scala new file mode 100644 index 000000000..13c3e3f60 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBOffsetStoreStateSpec.scala @@ -0,0 +1,113 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.Instant + +import akka.persistence.query.TimestampOffset +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Pid +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Record +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.SeqNr +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.State +import org.scalatest.TestSuite +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpec + +class DynamoDBOffsetStoreStateSpec extends AnyWordSpec with TestSuite with Matchers { + + def createRecord(pid: Pid, seqNr: SeqNr, timestamp: Instant): Record = { + Record(slice(pid), pid, seqNr, timestamp) + } + + def slice(pid: Pid): Int = math.abs(pid.hashCode % 1024) + + "DynamoDBOffsetStore.State" should { + "add records and keep track of pids and latest offset" in { + val t0 = TestClock.nowMillis().instant() + val state1 = State.empty + .add( + Vector( + createRecord("p1", 1, t0), + createRecord("p1", 2, t0.plusMillis(1)), + createRecord("p1", 3, t0.plusMillis(2)))) + state1.byPid("p1").seqNr shouldBe 3L + state1.offsetBySlice(slice("p1")) shouldBe TimestampOffset(t0.plusMillis(2), Map("p1" -> 3L)) + state1.latestTimestamp shouldBe t0.plusMillis(2) + state1.oldestTimestamp shouldBe t0 + + val state2 = state1.add(Vector(createRecord("p2", 2, t0.plusMillis(1)))) + state2.byPid("p1").seqNr shouldBe 3L + state2.byPid("p2").seqNr shouldBe 2L + slice("p2") should not be slice("p1") + state2.offsetBySlice(slice("p2")) shouldBe TimestampOffset(t0.plusMillis(1), Map("p2" -> 2L)) + // latest not updated because timestamp of p2 was before latest + state2.latestTimestamp shouldBe t0.plusMillis(2) + state2.oldestTimestamp shouldBe t0 + + val state3 = state2.add(Vector(createRecord("p3", 10, t0.plusMillis(3)))) + state3.byPid("p1").seqNr shouldBe 3L + state3.byPid("p2").seqNr shouldBe 2L + state3.byPid("p3").seqNr shouldBe 10L + slice("p3") should not be slice("p1") + slice("p3") should not be slice("p2") + state3.offsetBySlice(slice("p3")) shouldBe TimestampOffset(t0.plusMillis(3), Map("p3" -> 10L)) + state3.latestTimestamp shouldBe t0.plusMillis(3) + state3.oldestTimestamp shouldBe t0 + + // same slice and same timestamp, keep both in seen + slice("p10084") shouldBe slice("p3") + val state4 = state3.add(Vector(createRecord("p10084", 9, t0.plusMillis(3)))) + state4.offsetBySlice(slice("p10084")) shouldBe TimestampOffset(t0.plusMillis(3), Map("p3" -> 10L, "p10084" -> 9)) + } + + "evict old" in { + // these pids have the same slice 645, otherwise it will keep one for each slice + val p1 = "p500" + val p2 = "p621" + val p3 = "p742" + val p4 = "p863" + val p5 = "p984" + + val t0 = TestClock.nowMillis().instant() + val state1 = State.empty + .add( + Vector( + createRecord(p1, 1, t0), + createRecord(p2, 2, t0.plusMillis(1)), + createRecord(p3, 3, t0.plusMillis(2)), + createRecord(p4, 4, t0.plusMillis(3)), + createRecord(p5, 5, t0.plusMillis(4)))) + state1.oldestTimestamp shouldBe t0 + state1.byPid + .map { case (pid, r) => pid -> r.seqNr } shouldBe Map(p1 -> 1L, p2 -> 2L, p3 -> 3L, p4 -> 4L, p5 -> 5L) + + val state2 = state1.evict(t0.plusMillis(2), keepNumberOfEntries = 1) + state2.oldestTimestamp shouldBe t0.plusMillis(2) + state2.byPid.map { case (pid, r) => pid -> r.seqNr } shouldBe Map(p3 -> 3L, p4 -> 4L, p5 -> 5L) + + // keep all + state1.evict(t0.plusMillis(2), keepNumberOfEntries = 100) shouldBe state1 + + // keep 4 + val state3 = state1.evict(t0.plusMillis(2), keepNumberOfEntries = 4) + state3.oldestTimestamp shouldBe t0.plusMillis(1) + state3.byPid.map { case (pid, r) => pid -> r.seqNr } shouldBe Map(p2 -> 2L, p3 -> 3L, p4 -> 4L, p5 -> 5L) + } + + "find duplicate" in { + val t0 = TestClock.nowMillis().instant() + val state = + State.empty.add( + Vector( + createRecord("p1", 1, t0), + createRecord("p2", 2, t0.plusMillis(1)), + createRecord("p3", 3, t0.plusMillis(2)))) + state.isDuplicate(createRecord("p1", 1, t0)) shouldBe true + state.isDuplicate(createRecord("p1", 2, t0.plusMillis(10))) shouldBe false + state.isDuplicate(createRecord("p2", 1, t0)) shouldBe true + state.isDuplicate(createRecord("p4", 4, t0.plusMillis(10))) shouldBe false + } + } +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetProjectionSpec.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetProjectionSpec.scala new file mode 100644 index 000000000..42ea24a6a --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetProjectionSpec.scala @@ -0,0 +1,1587 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.Instant +import java.time.{ Duration => JDuration } +import java.util.UUID +import java.util.concurrent.CompletionException +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.atomic.AtomicInteger +import java.util.{ HashMap => JHashMap } + +import scala.annotation.tailrec +import scala.collection.immutable +import scala.concurrent.Await +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.jdk.CollectionConverters._ +import scala.jdk.FutureConverters._ +import scala.util.Failure +import scala.util.Success + +import akka.Done +import akka.NotUsed +import akka.actor.testkit.typed.TestException +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.actor.typed.ActorRef +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.internal.EnvelopeOrigin +import akka.persistence.query.Offset +import akka.persistence.query.TimestampOffset +import akka.persistence.query.TimestampOffsetBySlice +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.query.typed.scaladsl.EventTimestampQuery +import akka.persistence.query.typed.scaladsl.LoadEventQuery +import akka.persistence.typed.PersistenceId +import akka.projection.BySlicesSourceProvider +import akka.projection.HandlerRecoveryStrategy +import akka.projection.ProjectionBehavior +import akka.projection.ProjectionContext +import akka.projection.ProjectionId +import akka.projection.TestStatusObserver +import akka.projection.TestStatusObserver.Err +import akka.projection.TestStatusObserver.OffsetProgress +import akka.projection.dynamodb.internal.DynamoDBOffsetStore +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Pid +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.SeqNr +import akka.projection.dynamodb.scaladsl.DynamoDBProjection +import akka.projection.dynamodb.scaladsl.DynamoDBTransactHandler +import akka.projection.scaladsl.Handler +import akka.projection.scaladsl.SourceProvider +import akka.projection.testkit.scaladsl.ProjectionTestKit +import akka.projection.testkit.scaladsl.TestSourceProvider +import akka.stream.scaladsl.FlowWithContext +import akka.stream.scaladsl.Source +import akka.stream.testkit.TestSubscriber +import org.scalatest.wordspec.AnyWordSpecLike +import org.slf4j.LoggerFactory +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition +import software.amazon.awssdk.services.dynamodb.model.AttributeValue +import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest +import software.amazon.awssdk.services.dynamodb.model.GetItemRequest +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement +import software.amazon.awssdk.services.dynamodb.model.KeyType +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput +import software.amazon.awssdk.services.dynamodb.model.Put +import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + +object DynamoDBTimestampOffsetProjectionSpec { + + final case class Envelope(id: String, seqNr: Long, message: String) + + /** + * This variant of TestStatusObserver is useful when the incoming envelope is the original akka projection + * EventBySliceEnvelope, but we want to assert on [[Envelope]]. The original [[EventEnvelope]] has too many params + * that are not so interesting for the test including the offset timestamp that would make the it harder to test. + */ + class DynamoDBTestStatusObserver( + statusProbe: ActorRef[TestStatusObserver.Status], + progressProbe: ActorRef[TestStatusObserver.OffsetProgress[Envelope]]) + extends TestStatusObserver[EventEnvelope[String]](statusProbe.ref) { + override def offsetProgress(projectionId: ProjectionId, envelope: EventEnvelope[String]): Unit = + progressProbe ! OffsetProgress( + Envelope(envelope.persistenceId, envelope.sequenceNr, envelope.eventOption.getOrElse("None"))) + + override def error( + projectionId: ProjectionId, + envelope: EventEnvelope[String], + cause: Throwable, + recoveryStrategy: HandlerRecoveryStrategy): Unit = + statusProbe ! Err( + Envelope(envelope.persistenceId, envelope.sequenceNr, envelope.eventOption.getOrElse("None")), + cause) + } + + class TestTimestampSourceProvider( + envelopes: immutable.IndexedSeq[EventEnvelope[String]], + testSourceProvider: TestSourceProvider[Offset, EventEnvelope[String]], + override val maxSlice: Int) + extends SourceProvider[Offset, EventEnvelope[String]] + with BySlicesSourceProvider + with EventTimestampQuery + with LoadEventQuery { + + override def source(offset: () => Future[Option[Offset]]): Future[Source[EventEnvelope[String], NotUsed]] = + testSourceProvider.source(offset) + + override def extractOffset(envelope: EventEnvelope[String]): Offset = + testSourceProvider.extractOffset(envelope) + + override def extractCreationTime(envelope: EventEnvelope[String]): Long = + testSourceProvider.extractCreationTime(envelope) + + override def minSlice: Int = 0 + + override def timestampOf(persistenceId: String, sequenceNr: Long): Future[Option[Instant]] = { + Future.successful(envelopes.collectFirst { + case env + if env.persistenceId == persistenceId && env.sequenceNr == sequenceNr && env.offset + .isInstanceOf[TimestampOffset] => + env.offset.asInstanceOf[TimestampOffset].timestamp + }) + } + + override def loadEnvelope[Event](persistenceId: String, sequenceNr: Long): Future[EventEnvelope[Event]] = { + envelopes.collectFirst { + case env if env.persistenceId == persistenceId && env.sequenceNr == sequenceNr => + env.asInstanceOf[EventEnvelope[Event]] + } match { + case Some(env) => Future.successful(env) + case None => + Future.failed( + new NoSuchElementException( + s"Event with persistenceId [$persistenceId] and sequenceNr [$sequenceNr] not found.")) + } + } + } + + // test model is as simple as a text that gets other string concatenated to it + case class ConcatStr(id: String, text: String) { + def concat(newMsg: String): ConcatStr = { + if (text == "") + copy(id, newMsg) + else + copy(text = text + "|" + newMsg) + } + } + + final case class TestRepository()(implicit ec: ExecutionContext) { + private val store = new ConcurrentHashMap[String, String]() + + private val logger = LoggerFactory.getLogger(this.getClass) + + def concatToText(id: String, payload: String): Future[Done] = { + val savedStrOpt = findById(id) + + savedStrOpt.flatMap { strOpt => + val newConcatStr = strOpt + .map { + _.concat(payload) + } + .getOrElse(ConcatStr(id, payload)) + + upsert(newConcatStr) + } + } + + def update(id: String, payload: String): Future[Done] = { + upsert(ConcatStr(id, payload)) + } + + def updateWithNullValue(id: String): Future[Done] = { + upsert(ConcatStr(id, null)) + } + + private def upsert(concatStr: ConcatStr): Future[Done] = { + logger.debug("TestRepository.upsert: [{}]", concatStr) + store.put(concatStr.id, concatStr.text) + Future.successful(Done) + } + + def findById(id: String): Future[Option[ConcatStr]] = { + logger.debug("TestRepository.findById: [{}]", id) + Future.successful(Option(store.get(id)).map(text => ConcatStr(id, text))) + } + + } + + object TestTable { + val name = "projection_spec" + + object Attributes { + val Id = "id" + val Payload = "payload" + } + + def create(client: DynamoDbAsyncClient, system: ActorSystem[_]): Future[Done] = { + import system.executionContext + + client.describeTable(DescribeTableRequest.builder().tableName(name).build()).asScala.transformWith { + case Success(_) => Future.successful(Done) // already exists + case Failure(_) => + val request = CreateTableRequest + .builder() + .tableName(name) + .keySchema(KeySchemaElement.builder().attributeName(Attributes.Id).keyType(KeyType.HASH).build()) + .attributeDefinitions( + AttributeDefinition.builder().attributeName(Attributes.Id).attributeType(ScalarAttributeType.S).build()) + .provisionedThroughput(ProvisionedThroughput.builder().readCapacityUnits(5L).writeCapacityUnits(5L).build()) + .build() + client + .createTable(request) + .asScala + .map(_ => Done) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + } + + def findById(id: String)(client: DynamoDbAsyncClient, system: ActorSystem[_]): Future[Option[ConcatStr]] = { + import system.executionContext + + client + .getItem( + GetItemRequest + .builder() + .tableName(name) + .consistentRead(true) + .key(Map(Attributes.Id -> AttributeValue.fromS(id)).asJava) + .build()) + .asScala + .map { response => + if (response.hasItem && response.item.containsKey(Attributes.Payload)) + Some(ConcatStr(id, response.item.get(Attributes.Payload).s())) + else None + } + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + } + +} + +class DynamoDBTimestampOffsetProjectionSpec + extends ScalaTestWithActorTestKit(TestConfig.config) + with AnyWordSpecLike + with TestDbLifecycle + with TestData + with LogCapturing { + import DynamoDBTimestampOffsetProjectionSpec._ + + override def typedSystem: ActorSystem[_] = system + private implicit val ec: ExecutionContext = system.executionContext + + private val logger = LoggerFactory.getLogger(getClass) + private def createOffsetStore( + projectionId: ProjectionId, + sourceProvider: TestTimestampSourceProvider): DynamoDBOffsetStore = + new DynamoDBOffsetStore(projectionId, Some(sourceProvider), system, settings, client) + private val projectionTestKit = ProjectionTestKit(system) + + private val repository = new TestRepository() + + override protected def beforeAll(): Unit = { + if (localDynamoDB) + Await.result(TestTable.create(client, system), 10.seconds) + super.beforeAll() + } + + def createSourceProvider( + envelopes: immutable.IndexedSeq[EventEnvelope[String]], + complete: Boolean = true): TestTimestampSourceProvider = { + val sp = + TestSourceProvider[Offset, EventEnvelope[String]](Source(envelopes), _.offset) + .withStartSourceFrom { + case (lastProcessedOffsetBySlice: TimestampOffsetBySlice, offset: TimestampOffset) => + // FIXME: should have the envelope slice to handle this properly + val lastProcessedOffset = lastProcessedOffsetBySlice.offsets.head._2 + offset.timestamp.isBefore(lastProcessedOffset.timestamp) || + (offset.timestamp == lastProcessedOffset.timestamp && offset.seen == lastProcessedOffset.seen) + case _ => false + } + .withAllowCompletion(complete) + + new TestTimestampSourceProvider(envelopes, sp, persistenceExt.numberOfSlices - 1) + } + + def createBacktrackingSourceProvider( + envelopes: immutable.IndexedSeq[EventEnvelope[String]], + complete: Boolean = true): TestTimestampSourceProvider = { + val sp = + TestSourceProvider[Offset, EventEnvelope[String]](Source(envelopes), _.offset) + .withStartSourceFrom { (_, _) => false } // include all + .withAllowCompletion(complete) + new TestTimestampSourceProvider(envelopes, sp, persistenceExt.numberOfSlices - 1) + } + + private def latestOffsetShouldBe(expected: Any)(implicit offsetStore: DynamoDBOffsetStore) = { + val expectedTimestampOffset = expected.asInstanceOf[TimestampOffset] + offsetStore.readOffset().futureValue + offsetStore.getState().latestOffset shouldBe expectedTimestampOffset + } + + private def offsetShouldBeEmpty()(implicit offsetStore: DynamoDBOffsetStore) = { + offsetStore.readOffset[TimestampOffsetBySlice]().futureValue shouldBe empty + } + + private def projectedValueShouldBe(expected: String)(implicit entityId: String) = { + val opt = repository.findById(entityId).futureValue.map(_.text) + opt shouldBe Some(expected) + } + + private def projectedTestValueShouldBe(expected: String)(implicit entityId: String) = { + val opt = TestTable.findById(entityId)(client, system).futureValue.map(_.text) + opt shouldBe Some(expected) + } + + // TODO: extract this to some utility + @tailrec private def eventuallyExpectError(sinkProbe: TestSubscriber.Probe[_]): Throwable = { + sinkProbe.expectNextOrError() match { + case Right(_) => eventuallyExpectError(sinkProbe) + case Left(exc) => exc + } + } + + private val concatHandlerFail4Msg = "fail on fourth envelope" + + class ConcatHandler(repository: TestRepository, failPredicate: EventEnvelope[String] => Boolean = _ => false) + extends Handler[EventEnvelope[String]] { + + private val logger = LoggerFactory.getLogger(getClass) + private val _attempts = new AtomicInteger() + def attempts: Int = _attempts.get + + override def process(envelope: EventEnvelope[String]): Future[Done] = { + if (failPredicate(envelope)) { + _attempts.incrementAndGet() + throw TestException(concatHandlerFail4Msg + s" after $attempts attempts") + } else { + logger.debug(s"handling {}", envelope) + repository.concatToText(envelope.persistenceId, envelope.event) + } + } + + } + + class TransactConcatHandler(failPredicate: EventEnvelope[String] => Boolean = _ => false) + extends DynamoDBTransactHandler[EventEnvelope[String]] { + + private val logger = LoggerFactory.getLogger(getClass) + private val _attempts = new AtomicInteger() + def attempts: Int = _attempts.get + + override def process(envelope: EventEnvelope[String]): Future[Iterable[TransactWriteItem]] = { + if (failPredicate(envelope)) { + _attempts.incrementAndGet() + throw TestException(concatHandlerFail4Msg + s" after $attempts attempts") + } else { + logger.debug(s"handling {}", envelope) + TestTable.findById(envelope.persistenceId)(client, system).map { current => + val newPayload = current.fold(envelope.event)(_.concat(envelope.event).text) + val attributes = new JHashMap[String, AttributeValue] + attributes.put(TestTable.Attributes.Id, AttributeValue.fromS(envelope.persistenceId)) + attributes.put(TestTable.Attributes.Payload, AttributeValue.fromS(newPayload)) + Seq(TransactWriteItem.builder().put(Put.builder().tableName(TestTable.name).item(attributes).build()).build()) + } + } + } + } + + class TransactGroupedConcatHandler( + probe: ActorRef[String], + failPredicate: EventEnvelope[String] => Boolean = _ => false) + extends DynamoDBTransactHandler[Seq[EventEnvelope[String]]] { + + private val _attempts = new AtomicInteger() + def attempts: Int = _attempts.get + + override def process(envelopes: Seq[EventEnvelope[String]]): Future[Iterable[TransactWriteItem]] = { + probe ! "called" + + // can only have one TransactWriteItem per key + val envelopesByPid = envelopes.groupBy(_.persistenceId) + + Future.sequence(envelopesByPid.map { + case (pid, pidEnvelopes) => + TestTable.findById(pid)(client, system).map { + current => + val newConcatStr = pidEnvelopes.foldLeft(current.getOrElse(ConcatStr(pid, ""))) { (acc, env) => + if (failPredicate(env)) { + _attempts.incrementAndGet() + throw TestException(concatHandlerFail4Msg + s" after $attempts attempts") + } + acc.concat(env.event) + } + + val attributes = new JHashMap[String, AttributeValue] + attributes.put(TestTable.Attributes.Id, AttributeValue.fromS(pid)) + attributes.put(TestTable.Attributes.Payload, AttributeValue.fromS(newConcatStr.text)) + TransactWriteItem.builder().put(Put.builder().tableName(TestTable.name).item(attributes).build()).build() + } + }) + } + } + + private val clock = TestClock.nowMicros() + def tick(): TestClock = { + clock.tick(JDuration.ofMillis(1)) + clock + } + + def createEnvelope(pid: Pid, seqNr: SeqNr, timestamp: Instant, event: String): EventEnvelope[String] = { + val entityType = PersistenceId.extractEntityType(pid) + val slice = persistenceExt.sliceForPersistenceId(pid) + EventEnvelope( + TimestampOffset(timestamp, timestamp.plusMillis(1000), Map(pid -> seqNr)), + pid, + seqNr, + event, + timestamp.toEpochMilli, + entityType, + slice) + } + + def backtrackingEnvelope(env: EventEnvelope[String]): EventEnvelope[String] = + new EventEnvelope[String]( + env.offset, + env.persistenceId, + env.sequenceNr, + eventOption = None, + env.timestamp, + env.eventMetadata, + env.entityType, + env.slice, + env.filtered, + source = EnvelopeOrigin.SourceBacktracking) + + def createEnvelopes(pid: Pid, numberOfEvents: Int): immutable.IndexedSeq[EventEnvelope[String]] = { + (1 to numberOfEvents).map { n => + createEnvelope(pid, n, tick().instant(), s"e$n") + } + } + + def createEnvelopesWithDuplicates(pid1: Pid, pid2: Pid): Vector[EventEnvelope[String]] = { + val startTime = TestClock.nowMicros().instant() + + Vector( + createEnvelope(pid1, 1, startTime, s"e1-1"), + createEnvelope(pid1, 2, startTime.plusMillis(1), s"e1-2"), + createEnvelope(pid2, 1, startTime.plusMillis(2), s"e2-1"), + createEnvelope(pid1, 3, startTime.plusMillis(4), s"e1-3"), + // pid1-3 is emitted before pid2-2 even though pid2-2 timestamp is earlier, + // from backtracking query previous events are emitted again, including the missing pid2-2 + createEnvelope(pid1, 1, startTime, s"e1-1"), + createEnvelope(pid1, 2, startTime.plusMillis(1), s"e1-2"), + createEnvelope(pid2, 1, startTime.plusMillis(2), s"e2-1"), + // now it pid2-2 is included + createEnvelope(pid2, 2, startTime.plusMillis(3), s"e2-2"), + createEnvelope(pid1, 3, startTime.plusMillis(4), s"e1-3"), + // and then some normal again + createEnvelope(pid1, 4, startTime.plusMillis(5), s"e1-4"), + createEnvelope(pid2, 3, startTime.plusMillis(6), s"e2-3")) + } + + def createEnvelopesUnknownSequenceNumbers(startTime: Instant, pid1: Pid, pid2: Pid): Vector[EventEnvelope[String]] = { + Vector( + createEnvelope(pid1, 1, startTime, s"e1-1"), + createEnvelope(pid1, 2, startTime.plusMillis(1), s"e1-2"), + createEnvelope(pid2, 1, startTime.plusMillis(2), s"e2-1"), + // pid2 seqNr 2 missing, will reject 3 + createEnvelope(pid2, 3, startTime.plusMillis(4), s"e2-3"), + createEnvelope(pid1, 3, startTime.plusMillis(5), s"e1-3"), + // pid1 seqNr 4 missing, will reject 5 + createEnvelope(pid1, 5, startTime.plusMillis(7), s"e1-5")) + } + + def createEnvelopesBacktrackingUnknownSequenceNumbers( + startTime: Instant, + pid1: Pid, + pid2: Pid): Vector[EventEnvelope[String]] = { + Vector( + // may also contain some duplicates + createEnvelope(pid1, 2, startTime.plusMillis(1), s"e1-2"), + createEnvelope(pid2, 2, startTime.plusMillis(3), s"e2-2"), + createEnvelope(pid2, 3, startTime.plusMillis(4), s"e2-3"), + createEnvelope(pid1, 3, startTime.plusMillis(5), s"e1-3"), + createEnvelope(pid1, 4, startTime.plusMillis(6), s"e1-4"), + createEnvelope(pid1, 5, startTime.plusMillis(7), s"e1-5"), + createEnvelope(pid2, 4, startTime.plusMillis(8), s"e2-4"), + createEnvelope(pid1, 6, startTime.plusMillis(9), s"e1-6")) + } + + def markAsFilteredEvent[A](env: EventEnvelope[A]): EventEnvelope[A] = { + new EventEnvelope[A]( + env.offset, + env.persistenceId, + env.sequenceNr, + env.eventOption, + env.timestamp, + env.eventMetadata, + env.entityType, + env.slice, + filtered = true, + env.source) + } + + "A DynamoDB at-least-once projection with TimestampOffset" must { + + "persist projection and offset" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.atLeastOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new ConcatHandler(repository)) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedValueShouldBe("e1|e2|e3|e4|e5|e6") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter duplicates" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.atLeastOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new ConcatHandler(repository)) + + projectionTestKit.run(projection) { + projectedValueShouldBe("e1-1|e1-2|e1-3|e1-4")(pid1) + projectedValueShouldBe("e2-1|e2-2|e2-3")(pid2) + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter out unknown sequence numbers" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val startTime = TestClock.nowMicros().instant() + val sourceProvider = new TestSourceProviderWithInput() + implicit val offsetStore: DynamoDBOffsetStore = + new DynamoDBOffsetStore(projectionId, Some(sourceProvider), system, settings, client) + + val projectionRef = spawn( + ProjectionBehavior(DynamoDBProjection + .atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => new ConcatHandler(repository)))) + val input = sourceProvider.input.futureValue + + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes1.foreach(input ! _) + + eventually { + projectedValueShouldBe("e1-1|e1-2|e1-3")(pid1) + projectedValueShouldBe("e2-1")(pid2) + } + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes2.foreach(input ! _) + + eventually { + projectedValueShouldBe("e1-1|e1-2|e1-3|e1-4|e1-5|e1-6")(pid1) + projectedValueShouldBe("e2-1|e2-2|e2-3|e2-4")(pid2) + } + + eventually { + latestOffsetShouldBe(envelopes2.last.offset) + } + projectionRef ! ProjectionBehavior.Stop + } + + "re-delivery inflight events after failure with retry recovery strategy" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val failOnce = new AtomicBoolean(true) + val failPredicate = (ev: EventEnvelope[String]) => { + // fail on first call for event 4, let it pass afterwards + ev.sequenceNr == 4 && failOnce.compareAndSet(true, false) + } + val bogusEventHandler = new ConcatHandler(repository, failPredicate) + + val projectionFailing = + DynamoDBProjection + .atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => bogusEventHandler) + .withSaveOffset(afterEnvelopes = 5, afterDuration = 2.seconds) + .withRecoveryStrategy(HandlerRecoveryStrategy.retryAndSkip(2, 10.millis)) + + offsetShouldBeEmpty() + projectionTestKit.run(projectionFailing) { + projectedValueShouldBe("e1|e2|e3|e4|e5|e6") + } + + bogusEventHandler.attempts shouldBe 1 + + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "be able to skip envelopes but still store offset" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.atLeastOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new ConcatHandler(repository)) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedValueShouldBe("e1|e2|e5") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "handle async projection" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val result = new StringBuffer() + + def handler(): Handler[EventEnvelope[String]] = new Handler[EventEnvelope[String]] { + override def process(envelope: EventEnvelope[String]): Future[Done] = { + Future + .successful { + result.append(envelope.event).append("|") + } + .map(_ => Done) + } + } + + val projection = + DynamoDBProjection.atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => handler()) + + projectionTestKit.run(projection) { + result.toString shouldBe "e1|e2|e3|e4|e5|e6|" + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "re-delivery inflight events after failure with retry recovery strategy for async projection" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val failOnce = new AtomicBoolean(true) + val failPredicate = (ev: EventEnvelope[String]) => { + // fail on first call for event 4, let it pass afterwards + ev.sequenceNr == 4 && failOnce.compareAndSet(true, false) + } + + val result = new StringBuffer() + def handler(): Handler[EventEnvelope[String]] = new Handler[EventEnvelope[String]] { + override def process(envelope: EventEnvelope[String]): Future[Done] = { + if (failPredicate(envelope)) { + throw TestException(s"failed to process event '${envelope.sequenceNr}'") + } else { + Future + .successful { + result.append(envelope.event).append("|") + } + .map(_ => Done) + } + } + } + + val projectionFailing = + DynamoDBProjection + .atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => handler()) + .withSaveOffset(afterEnvelopes = 5, afterDuration = 2.seconds) + .withRecoveryStrategy(HandlerRecoveryStrategy.retryAndSkip(2, 10.millis)) + + offsetShouldBeEmpty() + projectionTestKit.run(projectionFailing) { + result.toString shouldBe "e1|e2|e3|e4|e5|e6|" + } + + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter duplicates for async projection" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val result1 = new StringBuffer() + val result2 = new StringBuffer() + + def handler(): Handler[EventEnvelope[String]] = new Handler[EventEnvelope[String]] { + override def process(envelope: EventEnvelope[String]): Future[Done] = { + Future + .successful { + if (envelope.persistenceId == pid1) + result1.append(envelope.event).append("|") + else + result2.append(envelope.event).append("|") + } + .map(_ => Done) + } + } + + val projection = + DynamoDBProjection.atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => handler()) + + projectionTestKit.run(projection) { + result1.toString shouldBe "e1-1|e1-2|e1-3|e1-4|" + result2.toString shouldBe "e2-1|e2-2|e2-3|" + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter out unknown sequence numbers for async projection" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val startTime = TestClock.nowMicros().instant() + val sourceProvider = new TestSourceProviderWithInput() + implicit val offsetStore: DynamoDBOffsetStore = + new DynamoDBOffsetStore(projectionId, Some(sourceProvider), system, settings, client) + + val result1 = new StringBuffer() + val result2 = new StringBuffer() + + def handler(): Handler[EventEnvelope[String]] = new Handler[EventEnvelope[String]] { + override def process(envelope: EventEnvelope[String]): Future[Done] = { + Future + .successful { + if (envelope.persistenceId == pid1) + result1.append(envelope.event).append("|") + else + result2.append(envelope.event).append("|") + } + .map(_ => Done) + } + } + + val projectionRef = spawn( + ProjectionBehavior( + DynamoDBProjection.atLeastOnce(projectionId, Some(settings), sourceProvider, handler = () => handler()))) + val input = sourceProvider.input.futureValue + + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes1.foreach(input ! _) + + eventually { + result1.toString shouldBe "e1-1|e1-2|e1-3|" + result2.toString shouldBe "e2-1|" + } + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes2.foreach(input ! _) + + eventually { + result1.toString shouldBe "e1-1|e1-2|e1-3|e1-4|e1-5|e1-6|" + result2.toString shouldBe "e2-1|e2-2|e2-3|e2-4|" + } + + eventually { + latestOffsetShouldBe(envelopes2.last.offset) + } + projectionRef ! ProjectionBehavior.Stop + } + + "be able to skip envelopes but still store offset for async projection" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.atLeastOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new ConcatHandler(repository)) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedValueShouldBe("e1|e2|e5") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + } + + "A DynamoDB exactly-once projection with TimestampOffset" must { + + "persist projection and offset in the same write operation (transactional)" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactConcatHandler) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1|e2|e3|e4|e5|e6") + } + latestOffsetShouldBe(envelopes.last.offset) + } + + "skip failing events when using RecoveryStrategy.skip" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val bogusEventHandler = new TransactConcatHandler(_.sequenceNr == 4) + + val projectionFailing = + DynamoDBProjection + .exactlyOnce(projectionId, Some(settings), sourceProvider, handler = () => bogusEventHandler) + .withRecoveryStrategy(HandlerRecoveryStrategy.skip) + + offsetShouldBeEmpty() + projectionTestKit.run(projectionFailing) { + projectedTestValueShouldBe("e1|e2|e3|e5|e6") + } + latestOffsetShouldBe(envelopes.last.offset) + } + + "store offset for failing events when using RecoveryStrategy.skip" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val bogusEventHandler = new TransactConcatHandler(_.sequenceNr == 6) + + val projectionFailing = + DynamoDBProjection + .exactlyOnce(projectionId, Some(settings), sourceProvider, handler = () => bogusEventHandler) + .withRecoveryStrategy(HandlerRecoveryStrategy.skip) + + offsetShouldBeEmpty() + projectionTestKit.run(projectionFailing) { + projectedTestValueShouldBe("e1|e2|e3|e4|e5") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "skip failing events after retrying when using RecoveryStrategy.retryAndSkip" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val statusProbe = createTestProbe[TestStatusObserver.Status]() + val progressProbe = createTestProbe[TestStatusObserver.OffsetProgress[Envelope]]() + val statusObserver = new DynamoDBTestStatusObserver(statusProbe.ref, progressProbe.ref) + val bogusEventHandler = new TransactConcatHandler(_.sequenceNr == 4) + + val projectionFailing = + DynamoDBProjection + .exactlyOnce(projectionId, Some(settings), sourceProvider, handler = () => bogusEventHandler) + .withRecoveryStrategy(HandlerRecoveryStrategy.retryAndSkip(3, 10.millis)) + .withStatusObserver(statusObserver) + + offsetShouldBeEmpty() + projectionTestKit.run(projectionFailing) { + projectedTestValueShouldBe("e1|e2|e3|e5|e6") + } + + // 1 + 3 => 1 original attempt and 3 retries + bogusEventHandler.attempts shouldBe 1 + 3 + + val someTestException = TestException("err") + statusProbe.expectMessage(TestStatusObserver.Err(Envelope(pid1, 4, "e4"), someTestException)) + statusProbe.expectMessage(TestStatusObserver.Err(Envelope(pid1, 4, "e4"), someTestException)) + statusProbe.expectMessage(TestStatusObserver.Err(Envelope(pid1, 4, "e4"), someTestException)) + statusProbe.expectMessage(TestStatusObserver.Err(Envelope(pid1, 4, "e4"), someTestException)) + statusProbe.expectNoMessage() + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 1, "e1"))) + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 2, "e2"))) + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 3, "e3"))) + // Offset 4 is stored even though it failed and was skipped + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 4, "e4"))) + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 5, "e5"))) + progressProbe.expectMessage(TestStatusObserver.OffsetProgress(Envelope(pid1, 6, "e6"))) + + latestOffsetShouldBe(envelopes.last.offset) + } + + "fail after retrying when using RecoveryStrategy.retryAndFail" in { + implicit val pid1 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid1, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val bogusEventHandler = new TransactConcatHandler(_.sequenceNr == 4) + + val projectionFailing = + DynamoDBProjection + .exactlyOnce(projectionId, Some(settings), sourceProvider, handler = () => bogusEventHandler) + .withRecoveryStrategy(HandlerRecoveryStrategy.retryAndFail(3, 10.millis)) + + offsetShouldBeEmpty() + projectionTestKit.runWithTestSink(projectionFailing) { sinkProbe => + sinkProbe.request(1000) + eventuallyExpectError(sinkProbe).getMessage should startWith(concatHandlerFail4Msg) + } + projectedTestValueShouldBe("e1|e2|e3") + // 1 + 3 => 1 original attempt and 3 retries + bogusEventHandler.attempts shouldBe 1 + 3 + + latestOffsetShouldBe(envelopes(2).offset) // <- offset is from e3 + } + + "restart from previous offset - fail with throwing an exception" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + def exactlyOnceProjection(failWhen: EventEnvelope[String] => Boolean = _ => false) = { + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactConcatHandler(failWhen)) + } + + offsetShouldBeEmpty() + projectionTestKit.runWithTestSink(exactlyOnceProjection(_.sequenceNr == 4)) { sinkProbe => + sinkProbe.request(1000) + eventuallyExpectError(sinkProbe).getMessage should startWith(concatHandlerFail4Msg) + } + projectedTestValueShouldBe("e1|e2|e3") + latestOffsetShouldBe(envelopes(2).offset) + + // re-run projection without failing function + projectionTestKit.run(exactlyOnceProjection()) { + projectedTestValueShouldBe("e1|e2|e3|e4|e5|e6") + } + latestOffsetShouldBe(envelopes.last.offset) + } + + "filter out duplicates" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactConcatHandler) + + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3|e1-4")(pid1) + projectedTestValueShouldBe("e2-1|e2-2|e2-3")(pid2) + } + latestOffsetShouldBe(envelopes.last.offset) + } + + "filter out unknown sequence numbers" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val startTime = TestClock.nowMicros().instant() + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + val sourceProvider1 = createSourceProvider(envelopes1) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider1) + + val projection1 = + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider1, + handler = () => new TransactConcatHandler) + + projectionTestKit.run(projection1) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3")(pid1) + projectedTestValueShouldBe("e2-1")(pid2) + } + latestOffsetShouldBe(envelopes1.collectFirst { case env if env.event == "e1-3" => env.offset }.get) + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + val sourceProvider2 = createBacktrackingSourceProvider(envelopes2) + val projection2 = + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider2, + handler = () => new TransactConcatHandler) + + projectionTestKit.run(projection2) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3|e1-4|e1-5|e1-6")(pid1) + projectedTestValueShouldBe("e2-1|e2-2|e2-3|e2-4")(pid2) + } + latestOffsetShouldBe(envelopes2.last.offset) + } + + "be able to skip envelopes but still store offset" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val projection = + DynamoDBProjection.exactlyOnce( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactConcatHandler) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1|e2|e5") + } + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "A DynamoDB grouped projection with TimestampOffset" must { + "persist projection and offset in the same write operation (transactional)" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val handlerProbe = createTestProbe[String]("calls-to-handler") + + val projection = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactGroupedConcatHandler(handlerProbe.ref)) + .withGroup(3, 3.seconds) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1|e2|e3|e4|e5|e6") + } + + // handler probe is called twice + handlerProbe.expectMessage("called") + handlerProbe.expectMessage("called") + + latestOffsetShouldBe(envelopes.last.offset) + } + + "filter duplicates" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val handlerProbe = createTestProbe[String]("calls-to-handler") + + val projection = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactGroupedConcatHandler(handlerProbe.ref)) + .withGroup(3, 3.seconds) + + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3|e1-4")(pid1) + projectedTestValueShouldBe("e2-1|e2-2|e2-3")(pid2) + } + + // handler probe is called twice + handlerProbe.expectMessage("called") + handlerProbe.expectMessage("called") + + latestOffsetShouldBe(envelopes.last.offset) + } + + "filter out unknown sequence numbers" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val startTime = TestClock.nowMicros().instant() + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + val sourceProvider1 = createBacktrackingSourceProvider(envelopes1) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider1) + + val handlerProbe = createTestProbe[String]() + + val projection1 = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider1, + handler = () => new TransactGroupedConcatHandler(handlerProbe.ref)) + .withGroup(3, 3.seconds) + + projectionTestKit.run(projection1) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3")(pid1) + projectedTestValueShouldBe("e2-1")(pid2) + } + latestOffsetShouldBe(envelopes1.collectFirst { case env if env.event == "e1-3" => env.offset }.get) + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + val sourceProvider2 = createBacktrackingSourceProvider(envelopes2) + val projection2 = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider2, + handler = () => new TransactGroupedConcatHandler(handlerProbe.ref)) + .withGroup(3, 3.seconds) + + projectionTestKit.run(projection2) { + projectedTestValueShouldBe("e1-1|e1-2|e1-3|e1-4|e1-5|e1-6")(pid1) + projectedTestValueShouldBe("e2-1|e2-2|e2-3|e2-4")(pid2) + } + latestOffsetShouldBe(envelopes2.last.offset) + } + + "be able to skip envelopes but still store offset" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val handlerProbe = createTestProbe[String]("calls-to-handler") + + val projection = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider, + handler = () => new TransactGroupedConcatHandler(handlerProbe.ref)) + .withGroup(3, 3.seconds) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + projectedTestValueShouldBe("e1|e2|e5") + } + + // handler probe is called twice + handlerProbe.expectMessage("called") + handlerProbe.expectMessage("called") + + latestOffsetShouldBe(envelopes.last.offset) + } + + "handle at-least-once grouped projection" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes1 = createEnvelopes(pid1, 3) + val envelopes2 = createEnvelopes(pid2, 3) + val envelopes = envelopes1.zip(envelopes2).flatMap { case (a, b) => List(a, b) } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val result = new StringBuffer() + + def handler(): Handler[immutable.Seq[EventEnvelope[String]]] = + new Handler[immutable.Seq[EventEnvelope[String]]] { + override def process(envelopes: immutable.Seq[EventEnvelope[String]]): Future[Done] = { + Future { + envelopes.foreach(env => result.append(env.persistenceId).append("_").append(env.event).append("|")) + }.map(_ => Done) + } + } + + val projection = + DynamoDBProjection + .atLeastOnceGroupedWithin(projectionId, Some(settings), sourceProvider, handler = () => handler()) + .withGroup(2, 3.seconds) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + result.toString shouldBe s"${pid1}_e1|${pid2}_e1|${pid1}_e2|${pid2}_e2|${pid1}_e3|${pid2}_e3|" + } + + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + offsetStore.storedSeqNr(pid1).futureValue shouldBe 3 + offsetStore.storedSeqNr(pid2).futureValue shouldBe 3 + } + + "filter duplicates for at-least-once grouped projection" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val result1 = new StringBuffer() + val result2 = new StringBuffer() + + def handler(): Handler[immutable.Seq[EventEnvelope[String]]] = new Handler[immutable.Seq[EventEnvelope[String]]] { + override def process(envelopes: immutable.Seq[EventEnvelope[String]]): Future[Done] = { + Future + .successful { + envelopes.foreach { envelope => + if (envelope.persistenceId == pid1) + result1.append(envelope.event).append("|") + else + result2.append(envelope.event).append("|") + } + } + .map(_ => Done) + } + } + + val projection = + DynamoDBProjection.atLeastOnceGroupedWithin( + projectionId, + Some(settings), + sourceProvider, + handler = () => handler()) + + projectionTestKit.run(projection) { + result1.toString shouldBe "e1-1|e1-2|e1-3|e1-4|" + result2.toString shouldBe "e2-1|e2-2|e2-3|" + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter out unknown sequence numbers for at-least-once grouped projection" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val startTime = TestClock.nowMicros().instant() + val sourceProvider = new TestSourceProviderWithInput() + implicit val offsetStore: DynamoDBOffsetStore = + new DynamoDBOffsetStore(projectionId, Some(sourceProvider), system, settings, client) + + val result1 = new StringBuffer() + val result2 = new StringBuffer() + + def handler(): Handler[immutable.Seq[EventEnvelope[String]]] = new Handler[immutable.Seq[EventEnvelope[String]]] { + override def process(envelopes: immutable.Seq[EventEnvelope[String]]): Future[Done] = { + Future + .successful { + envelopes.foreach { envelope => + if (envelope.persistenceId == pid1) + result1.append(envelope.event).append("|") + else + result2.append(envelope.event).append("|") + } + } + .map(_ => Done) + } + } + + val projectionRef = spawn( + ProjectionBehavior(DynamoDBProjection + .atLeastOnceGroupedWithin(projectionId, Some(settings), sourceProvider, handler = () => handler()))) + val input = sourceProvider.input.futureValue + + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes1.foreach(input ! _) + + eventually { + result1.toString shouldBe "e1-1|e1-2|e1-3|" + result2.toString shouldBe "e2-1|" + } + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes2.foreach(input ! _) + + eventually { + result1.toString shouldBe "e1-1|e1-2|e1-3|e1-4|e1-5|e1-6|" + result2.toString shouldBe "e2-1|e2-2|e2-3|e2-4|" + } + + eventually { + latestOffsetShouldBe(envelopes2.last.offset) + } + projectionRef ! ProjectionBehavior.Stop + } + + "be able to skip envelopes but still store offset for at-least-once grouped projection" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + val result = new StringBuffer() + + def handler(): Handler[immutable.Seq[EventEnvelope[String]]] = + new Handler[immutable.Seq[EventEnvelope[String]]] { + override def process(envelopes: immutable.Seq[EventEnvelope[String]]): Future[Done] = { + Future { + envelopes.foreach(env => result.append(env.event).append("|")) + }.map(_ => Done) + } + } + + val projection = + DynamoDBProjection + .atLeastOnceGroupedWithin(projectionId, Some(settings), sourceProvider, handler = () => handler()) + .withGroup(2, 3.seconds) + + offsetShouldBeEmpty() + projectionTestKit.run(projection) { + result.toString shouldBe "e1|e2|e5|" + } + + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + } + + "A DynamoDB flow projection with TimestampOffset" must { + + "persist projection and offset" in { + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + offsetShouldBeEmpty() + + val flowHandler = + FlowWithContext[EventEnvelope[String], ProjectionContext] + .mapAsync(1) { env => + repository.concatToText(env.persistenceId, env.event) + } + + val projection = + DynamoDBProjection + .atLeastOnceFlow(projectionId, Some(settings), sourceProvider, handler = flowHandler) + .withSaveOffset(1, 1.minute) + + projectionTestKit.run(projection) { + projectedValueShouldBe("e1|e2|e3|e4|e5|e6") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter duplicates" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopesWithDuplicates(pid1, pid2) + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + info(s"pid1 [$pid1], pid2 [$pid2]") + + val flowHandler = + FlowWithContext[EventEnvelope[String], ProjectionContext] + .mapAsync(1) { env => + repository.concatToText(env.persistenceId, env.event) + } + + val projection = + DynamoDBProjection + .atLeastOnceFlow(projectionId, Some(settings), sourceProvider, handler = flowHandler) + .withSaveOffset(2, 1.minute) + + projectionTestKit.run(projection) { + projectedValueShouldBe("e1-1|e1-2|e1-3|e1-4")(pid1) + projectedValueShouldBe("e2-1|e2-2|e2-3")(pid2) + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + + "filter out unknown sequence numbers" in { + val pid1 = UUID.randomUUID().toString + val pid2 = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + + val startTime = TestClock.nowMicros().instant() + val sourceProvider = new TestSourceProviderWithInput() + implicit val offsetStore: DynamoDBOffsetStore = + new DynamoDBOffsetStore(projectionId, Some(sourceProvider), system, settings, client) + + val flowHandler = + FlowWithContext[EventEnvelope[String], ProjectionContext] + .mapAsync(1) { env => + repository.concatToText(env.persistenceId, env.event) + } + + val projectionRef = spawn( + ProjectionBehavior( + DynamoDBProjection + .atLeastOnceFlow(projectionId, Some(settings), sourceProvider, handler = flowHandler) + .withSaveOffset(2, 1.minute))) + val input = sourceProvider.input.futureValue + + val envelopes1 = createEnvelopesUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes1.foreach(input ! _) + + eventually { + projectedValueShouldBe("e1-1|e1-2|e1-3")(pid1) + projectedValueShouldBe("e2-1")(pid2) + } + + // simulate backtracking + logger.debug("Starting backtracking") + val envelopes2 = createEnvelopesBacktrackingUnknownSequenceNumbers(startTime, pid1, pid2) + envelopes2.foreach(input ! _) + + eventually { + projectedValueShouldBe("e1-1|e1-2|e1-3|e1-4|e1-5|e1-6")(pid1) + projectedValueShouldBe("e2-1|e2-2|e2-3|e2-4")(pid2) + } + + eventually { + latestOffsetShouldBe(envelopes2.last.offset) + } + projectionRef ! ProjectionBehavior.Stop + } + + "not support skipping envelopes but still store offset" in { + // This is a limitation for atLeastOnceFlow and this test is just + // capturing current behavior of not supporting the feature of skipping + // envelopes that are marked with NotUsed in the eventMetadata. + implicit val pid = UUID.randomUUID().toString + val projectionId = genRandomProjectionId() + val envelopes = createEnvelopes(pid, 6).map { env => + if (env.event == "e3" || env.event == "e4" || env.event == "e6") + markAsFilteredEvent(env) + else + env + } + val sourceProvider = createSourceProvider(envelopes) + implicit val offsetStore = createOffsetStore(projectionId, sourceProvider) + + offsetShouldBeEmpty() + + val flowHandler = + FlowWithContext[EventEnvelope[String], ProjectionContext] + .mapAsync(1) { env => + repository.concatToText(env.persistenceId, env.event) + } + + val projection = + DynamoDBProjection + .atLeastOnceFlow(projectionId, Some(settings), sourceProvider, handler = flowHandler) + .withSaveOffset(1, 1.minute) + + projectionTestKit.run(projection) { + // e3, e4, e6 are still included + projectedValueShouldBe("e1|e2|e3|e4|e5|e6") + } + eventually { + latestOffsetShouldBe(envelopes.last.offset) + } + } + } + + // FIXME see more tests in R2dbcTimestampOffsetProjectionSpec + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetStoreSpec.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetStoreSpec.scala new file mode 100644 index 000000000..c148ca620 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/DynamoDBTimestampOffsetStoreSpec.scala @@ -0,0 +1,1061 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.Instant +import java.time.{ Duration => JDuration } +import java.util.UUID + +import scala.concurrent.Await +import scala.concurrent.Future +import scala.concurrent.duration._ + +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.internal.EnvelopeOrigin +import akka.persistence.query.TimestampOffset +import akka.persistence.query.TimestampOffsetBySlice +import akka.persistence.query.UpdatedDurableState +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.query.typed.scaladsl.EventTimestampQuery +import akka.persistence.query.typed.scaladsl.LoadEventQuery +import akka.persistence.typed.PersistenceId +import akka.projection.BySlicesSourceProvider +import akka.projection.ProjectionId +import akka.projection.dynamodb.internal.DynamoDBOffsetStore +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Pid +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.SeqNr +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Validation.Accepted +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Validation.Duplicate +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Validation.RejectedBacktrackingSeqNr +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Validation.RejectedSeqNr +import akka.projection.dynamodb.internal.OffsetPidSeqNr +import akka.projection.dynamodb.internal.OffsetStoreDao.OffsetStoreAttributes +import akka.projection.internal.ManagementState +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import org.scalatest.OptionValues +import org.scalatest.wordspec.AnyWordSpecLike +import org.slf4j.LoggerFactory + +object DynamoDBTimestampOffsetStoreSpec { + val config: Config = + ConfigFactory + .parseString(""" + # to be able to test eviction + akka.projection.dynamodb.offset-store.keep-number-of-entries = 0 + """) + .withFallback(TestConfig.config) + + def configWithOffsetTTL: Config = + ConfigFactory + .parseString(""" + akka.projection.dynamodb.time-to-live { + projections { + "*" { + offset-time-to-live = 1 hour + } + } + } + """) + .withFallback(config) + + class TestTimestampSourceProvider(override val minSlice: Int, override val maxSlice: Int, clock: TestClock) + extends BySlicesSourceProvider + with EventTimestampQuery + with LoadEventQuery { + + override def timestampOf(persistenceId: String, sequenceNr: SeqNr): Future[Option[Instant]] = + Future.successful(Some(clock.instant())) + + override def loadEnvelope[Event](persistenceId: String, sequenceNr: SeqNr): Future[EventEnvelope[Event]] = + throw new IllegalStateException("loadEvent shouldn't be used here") + } +} + +class DynamoDBTimestampOffsetStoreSpec + extends DynamoDBTimestampOffsetStoreBaseSpec(DynamoDBTimestampOffsetStoreSpec.config) + +class DynamoDBTimestampOffsetStoreWithOffsetTTLSpec + extends DynamoDBTimestampOffsetStoreBaseSpec(DynamoDBTimestampOffsetStoreSpec.configWithOffsetTTL) { + override protected def usingOffsetTTL: Boolean = true +} + +abstract class DynamoDBTimestampOffsetStoreBaseSpec(config: Config) + extends ScalaTestWithActorTestKit(config) + with AnyWordSpecLike + with TestDbLifecycle + with TestData + with OptionValues + with LogCapturing { + import DynamoDBTimestampOffsetStoreSpec.TestTimestampSourceProvider + + override def typedSystem: ActorSystem[_] = system + + private val clock = TestClock.nowMicros() + def tick(): Unit = clock.tick(JDuration.ofMillis(1)) + + private val log = LoggerFactory.getLogger(getClass) + + protected def usingOffsetTTL: Boolean = false + + private def createOffsetStore( + projectionId: ProjectionId, + customSettings: DynamoDBProjectionSettings = settings, + eventTimestampQueryClock: TestClock = clock) = + new DynamoDBOffsetStore( + projectionId, + Some(new TestTimestampSourceProvider(0, persistenceExt.numberOfSlices - 1, eventTimestampQueryClock)), + system, + customSettings, + client) + + def createEnvelope(pid: Pid, seqNr: SeqNr, timestamp: Instant, event: String): EventEnvelope[String] = { + val entityType = PersistenceId.extractEntityType(pid) + val slice = persistenceExt.sliceForPersistenceId(pid) + EventEnvelope( + TimestampOffset(timestamp, timestamp.plusMillis(1000), Map(pid -> seqNr)), + pid, + seqNr, + event, + timestamp.toEpochMilli, + entityType, + slice) + } + + def backtrackingEnvelope(env: EventEnvelope[String]): EventEnvelope[String] = + new EventEnvelope[String]( + env.offset, + env.persistenceId, + env.sequenceNr, + eventOption = None, + env.timestamp, + env.eventMetadata, + env.entityType, + env.slice, + env.filtered, + source = EnvelopeOrigin.SourceBacktracking) + + def filteredEnvelope(env: EventEnvelope[String]): EventEnvelope[String] = + new EventEnvelope[String]( + env.offset, + env.persistenceId, + env.sequenceNr, + env.eventOption, + env.timestamp, + env.eventMetadata, + env.entityType, + env.slice, + filtered = true, + env.source) + + def createUpdatedDurableState( + pid: Pid, + revision: SeqNr, + timestamp: Instant, + state: String): UpdatedDurableState[String] = + new UpdatedDurableState( + pid, + revision, + state, + TimestampOffset(timestamp, timestamp.plusMillis(1000), Map(pid -> revision)), + timestamp.toEpochMilli) + + def slice(pid: String): Int = + persistenceExt.sliceForPersistenceId(pid) + + s"The DynamoDBOffsetStore for TimestampOffset" must { + + "save TimestampOffset with one entry" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + tick() + val offset1 = TimestampOffset(clock.instant(), Map("p1" -> 3L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + val readOffset1 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + readOffset1.get.offsets(slice("p1")) shouldBe offset1 + offsetStore.getState().offsetBySlice(slice("p1")) shouldBe offset1 + offsetStore.storedSeqNr("p1").futureValue shouldBe 3L + + tick() + val offset2 = TimestampOffset(clock.instant(), Map("p1" -> 4L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset2, "p1", 4L)).futureValue + val readOffset2 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + readOffset2.get.offsets(slice("p1")) shouldBe offset2 // yep, saveOffset overwrites previous + offsetStore.getState().offsetBySlice(slice("p1")) shouldBe offset2 + offsetStore.storedSeqNr("p1").futureValue shouldBe 4L + + val timestampOffsetItem = getOffsetItemFor(projectionId, slice("p1")).value + val seqNrOffsetItem = getOffsetItemFor(projectionId, slice("p1"), "p1").value + + if (usingOffsetTTL) { + val expected = System.currentTimeMillis / 1000 + 1.hour.toSeconds + val timestampOffsetExpiry = timestampOffsetItem.get(OffsetStoreAttributes.Expiry).value.n.toLong + timestampOffsetExpiry should (be <= expected and be > expected - 10) // within 10 seconds + val seqNrOffsetExpiry = seqNrOffsetItem.get(OffsetStoreAttributes.Expiry).value.n.toLong + seqNrOffsetExpiry should (be <= expected and be > expected - 10) // within 10 seconds + } else { + timestampOffsetItem.get(OffsetStoreAttributes.Expiry) shouldBe None + seqNrOffsetItem.get(OffsetStoreAttributes.Expiry) shouldBe None + } + } + + "save TimestampOffset with several seen entries" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + val entityType = nextEntityType() + val p1 = nextPersistenceId(entityType) + val s = slice(p1.id) + val p2 = randomPersistenceIdForSlice(entityType, s) + val p3 = randomPersistenceIdForSlice(entityType, s) + val p4 = randomPersistenceIdForSlice(entityType, s) + + tick() + val offset1a = TimestampOffset(clock.instant(), Map(p1.id -> 3L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1a, p1.id, 3L)).futureValue + val offset1b = TimestampOffset(clock.instant(), Map(p1.id -> 3L, p2.id -> 1L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1b, p2.id, 1L)).futureValue + val offset1c = TimestampOffset(clock.instant(), Map(p1.id -> 3L, p2.id -> 1L, p3.id -> 5L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1c, p3.id, 5L)).futureValue + + val readOffset1 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + val expectedOffset1 = + TimestampOffset(offset1a.timestamp, offset1a.readTimestamp, Map(p1.id -> 3L, p2.id -> 1L, p3.id -> 5L)) + readOffset1.get.offsets(s) shouldBe expectedOffset1 + + tick() + val offset2 = TimestampOffset(clock.instant(), Map(p1.id -> 4L, p3.id -> 6L, p4.id -> 9L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset2, p3.id, 6L)).futureValue + val readOffset2 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + // note that it's not the seen Map in the saveOffset that is used, but the pid, seqNr of saveOffset, + // so here we have only saved p3 -> 6 + val expectedOffset2 = TimestampOffset(offset2.timestamp, offset2.readTimestamp, Map(p3.id -> 6L)) + readOffset2.get.offsets(s) shouldBe expectedOffset2 + } + + "save TimestampOffset when same timestamp" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + val entityType = nextEntityType() + val p1 = nextPersistenceId(entityType) + val s = slice(p1.id) + val p2 = randomPersistenceIdForSlice(entityType, s) + val p3 = randomPersistenceIdForSlice(entityType, s) + val p4 = randomPersistenceIdForSlice(entityType, s) + + tick() + // the seen map in saveOffset is not used when saving, so using empty Map for simplicity + val offset1 = TimestampOffset(clock.instant(), Map.empty) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, p1.id, 3L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, p2.id, 1L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, p3.id, 5L)).futureValue + offsetStore.readOffset().futureValue + val expectedOffset1 = TimestampOffset(clock.instant(), Map(p1.id -> 3L, p2.id -> 1L, p3.id -> 5L)) + offsetStore.getState().offsetBySlice(s) shouldBe expectedOffset1 + + // not tick, same timestamp + val offset2 = TimestampOffset(clock.instant(), Map.empty) + offsetStore.saveOffset(OffsetPidSeqNr(offset2, p2.id, 2L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset2, p4.id, 9L)).futureValue + offsetStore.readOffset().futureValue + val expectedOffset2 = TimestampOffset(clock.instant(), expectedOffset1.seen ++ Map(p2.id -> 2L, p4.id -> 9L)) + // all should be included since same timestamp + offsetStore.getState().offsetBySlice(s) shouldBe expectedOffset2 + + // saving new with later timestamp + tick() + val offset3 = TimestampOffset(clock.instant(), Map(p1.id -> 4L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset3, p1.id, 4L)).futureValue + offsetStore.readOffset().futureValue + offsetStore.getState().offsetBySlice(s) shouldBe offset3 + } + + "save batch of TimestampOffsets" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + val p1 = "p1" + val slice1 = persistenceExt.sliceForPersistenceId(p1) + slice1 shouldBe 449 + + val p2 = "p2" + val slice2 = persistenceExt.sliceForPersistenceId(p2) + slice2 shouldBe 450 + + val p3 = "p10" + val slice3 = persistenceExt.sliceForPersistenceId(p3) + slice3 shouldBe 655 + + val p4 = "p11" + val slice4 = persistenceExt.sliceForPersistenceId(p4) + slice4 shouldBe 656 + + tick() + val offset1 = TimestampOffset(clock.instant(), Map.empty) + tick() + val offset2 = TimestampOffset(clock.instant(), Map.empty) + tick() + val offset3 = TimestampOffset(clock.instant(), Map.empty) + val offsetsBatch1 = Vector( + OffsetPidSeqNr(offset1, p1, 3L), + OffsetPidSeqNr(offset1, p2, 1L), + OffsetPidSeqNr(offset1, p3, 5L), + OffsetPidSeqNr(offset2, p4, 1L), + OffsetPidSeqNr(offset2, p1, 4L), + OffsetPidSeqNr(offset3, p2, 2L)) + + offsetStore.saveOffsets(offsetsBatch1).futureValue + val readOffset1 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + offsetStore.getState().offsetBySlice(slice1) shouldBe TimestampOffset(offset2.timestamp, Map(p1 -> 4L)) + offsetStore.getState().offsetBySlice(slice2) shouldBe TimestampOffset(offset3.timestamp, Map(p2 -> 2L)) + offsetStore.getState().offsetBySlice(slice3) shouldBe TimestampOffset(offset1.timestamp, Map(p3 -> 5L)) + offsetStore.getState().offsetBySlice(slice4) shouldBe TimestampOffset(offset2.timestamp, Map(p4 -> 1L)) + readOffset1.get.offsets shouldBe offsetStore.getState().offsetBySlice + + val state1 = offsetStore.load(Vector(p1, p2, p3, p4)).futureValue + state1.byPid(p1).seqNr shouldBe 4L + state1.byPid(p2).seqNr shouldBe 2L + state1.byPid(p3).seqNr shouldBe 5L + state1.byPid(p4).seqNr shouldBe 1L + + tick() + val offset5 = TimestampOffset(clock.instant(), Map(p1 -> 5L)) + offsetStore.saveOffsets(Vector(OffsetPidSeqNr(offset5, p1, 5L))).futureValue + + tick() + // duplicate + val offset6 = TimestampOffset(clock.instant(), Map(p2 -> 1L)) + offsetStore.saveOffsets(Vector(OffsetPidSeqNr(offset6, p2, 1L))).futureValue + + tick() + val offset7 = TimestampOffset(clock.instant(), Map(p1 -> 6L)) + tick() + val offset8 = TimestampOffset(clock.instant(), Map(p1 -> 7L)) + tick() + val offset9 = TimestampOffset(clock.instant(), Map(p1 -> 8L)) + val offsetsBatch2 = + Vector(OffsetPidSeqNr(offset7, p1, 6L), OffsetPidSeqNr(offset8, p1, 7L), OffsetPidSeqNr(offset9, p1, 8L)) + + offsetStore.saveOffsets(offsetsBatch2).futureValue + val readOffset2 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + val state2 = offsetStore.load(Vector(p1, p2, p3, p4)).futureValue + state2.byPid(p1).seqNr shouldBe 8L + state2.byPid(p2).seqNr shouldBe 2L // duplicate with lower seqNr not saved + state2.byPid(p3).seqNr shouldBe 5L + state2.byPid(p4).seqNr shouldBe 1L + readOffset2.get.offsets shouldBe offsetStore.getState().offsetBySlice + + for (pid <- Seq(p1, p2, p3, p4)) { + val timestampOffsetItem = getOffsetItemFor(projectionId, slice(pid)).value + val seqNrOffsetItem = getOffsetItemFor(projectionId, slice(pid), pid).value + + if (usingOffsetTTL) { + val expected = System.currentTimeMillis / 1000 + 1.hour.toSeconds + val timestampOffsetExpiry = timestampOffsetItem.get(OffsetStoreAttributes.Expiry).value.n.toLong + timestampOffsetExpiry should (be <= expected and be > expected - 10) // within 10 seconds + val seqNrOffsetExpiry = seqNrOffsetItem.get(OffsetStoreAttributes.Expiry).value.n.toLong + seqNrOffsetExpiry should (be <= expected and be > expected - 10) // within 10 seconds + } else { + timestampOffsetItem.get(OffsetStoreAttributes.Expiry) shouldBe None + seqNrOffsetItem.get(OffsetStoreAttributes.Expiry) shouldBe None + } + } + } + + "save batch of many TimestampOffsets" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + def test(pidPrefix: String, numberOfOffsets: Int): Unit = { + withClue(s"with $numberOfOffsets offsets: ") { + val offsetsBatch = (1 to numberOfOffsets).map { n => + tick() + val offset = TimestampOffset(clock.instant(), Map.empty) + OffsetPidSeqNr(offset, s"$pidPrefix$n", n) + } + offsetStore.saveOffsets(offsetsBatch).futureValue + offsetStore.readOffset().futureValue + (1 to numberOfOffsets).map { n => + val pid = s"$pidPrefix$n" + val state = offsetStore.load(pid).futureValue + state.byPid(pid).seqNr shouldBe n + } + } + } + + test("a", settings.offsetBatchSize) + test("a", settings.offsetBatchSize - 1) + test("a", settings.offsetBatchSize + 1) + test("a", settings.offsetBatchSize * 2) + test("a", settings.offsetBatchSize * 2 - 1) + test("a", settings.offsetBatchSize * 2 + 1) + } + + "perf save batch of TimestampOffsets" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + val warmupIterations = 1 // increase this for serious testing + val iterations = 2000 // increase this for serious testing + val batchSize = 100 + + // warmup + (1 to warmupIterations).foreach { _ => + val offsets = (1 to batchSize).map { n => + val offset = TimestampOffset(Instant.now(), Map(s"p$n" -> 1L)) + OffsetPidSeqNr(offset, s"p$n", 1L) + } + Await.result(offsetStore.saveOffsets(offsets), 5.seconds) + } + + val totalStartTime = System.nanoTime() + var startTime = System.nanoTime() + var count = 0 + + (1 to iterations).foreach { i => + val offsets = (1 to batchSize).map { n => + val offset = TimestampOffset(Instant.now(), Map(s"p$n" -> 1L)) + OffsetPidSeqNr(offset, s"p$n", 1L) + } + count += batchSize + Await.result(offsetStore.saveOffsets(offsets), 5.seconds) + + if (i % 1000 == 0) { + val totalDurationMs = (System.nanoTime() - totalStartTime) / 1000 / 1000 + val durationMs = (System.nanoTime() - startTime) / 1000 / 1000 + log.debug( + s"#${i * batchSize}: $count took $durationMs ms, RPS ${1000L * count / durationMs}, Total RPS ${1000L * i * batchSize / totalDurationMs}") + startTime = System.nanoTime() + count = 0 + } + } + } + + "not update when earlier seqNr" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + tick() + val offset1 = TimestampOffset(clock.instant(), Map("p1" -> 3L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + val readOffset1 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + readOffset1.get.offsets(slice("p1")) shouldBe offset1 + + clock.setInstant(clock.instant().minusMillis(1)) + val offset2 = TimestampOffset(clock.instant(), Map("p1" -> 2L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset2, "p1", 2L)).futureValue + val readOffset2 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + readOffset2.get.offsets(slice("p1")) shouldBe offset1 // keeping offset1 + } + + "readOffset from given slices" in { + val projectionId0 = ProjectionId(UUID.randomUUID().toString, "0-1023") + val projectionId1 = ProjectionId(projectionId0.name, "0-511") + val projectionId2 = ProjectionId(projectionId0.name, "512-1023") + + val p1 = "p1" + val slice1 = persistenceExt.sliceForPersistenceId(p1) + slice1 shouldBe 449 + + val p2 = "p2" + val slice2 = persistenceExt.sliceForPersistenceId(p2) + slice2 shouldBe 450 + + val p3 = "p10" + val slice3 = persistenceExt.sliceForPersistenceId(p3) + slice3 shouldBe 655 + + val p4 = "p11" + val slice4 = persistenceExt.sliceForPersistenceId(p4) + slice4 shouldBe 656 + + val offsetStore0 = + new DynamoDBOffsetStore( + projectionId0, + Some(new TestTimestampSourceProvider(0, persistenceExt.numberOfSlices - 1, clock)), + system, + settings, + client) + + tick() + val offset1 = TimestampOffset(clock.instant(), Map(p1 -> 3L)) + offsetStore0.saveOffset(OffsetPidSeqNr(offset1, p1, 3L)).futureValue + tick() + val offset2 = TimestampOffset(clock.instant(), Map(p2 -> 4L)) + offsetStore0.saveOffset(OffsetPidSeqNr(offset2, p2, 4L)).futureValue + tick() + val offset3 = TimestampOffset(clock.instant(), Map(p3 -> 7L)) + offsetStore0.saveOffset(OffsetPidSeqNr(offset3, p3, 7L)).futureValue + tick() + val offset4 = TimestampOffset(clock.instant(), Map(p4 -> 5L)) + offsetStore0.saveOffset(OffsetPidSeqNr(offset4, p4, 5L)).futureValue + + val offsetStore1 = + new DynamoDBOffsetStore( + projectionId1, + Some(new TestTimestampSourceProvider(0, 511, clock)), + system, + settings, + client) + offsetStore1.readOffset().futureValue + // FIXME this is not really testing anything, the test is supposed to test that it is responsible for a range + val state1 = offsetStore1.load(Vector(p1, p2)).futureValue + state1.byPid.keySet shouldBe Set(p1, p2) + + val offsetStore2 = + new DynamoDBOffsetStore( + projectionId2, + Some(new TestTimestampSourceProvider(512, 1023, clock)), + system, + settings, + client) + offsetStore2.readOffset().futureValue + // FIXME this is not really testing anything, the test is supposed to test that it is responsible for a range + val state2 = offsetStore2.load(Vector(p3, p4)).futureValue + state2.byPid.keySet shouldBe Set(p3, p4) + } + + "filter duplicates" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + tick() + val offset1 = TimestampOffset(clock.instant(), Map("p1" -> 3L, "p2" -> 1L, "p3" -> 5L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p2", 1L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p3", 5L)).futureValue + tick() + val offset2 = TimestampOffset(clock.instant(), Map("p1" -> 4L, "p3" -> 6L, "p4" -> 9L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset2, "p1", 4L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset2, "p3", 6L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset2, "p4", 9L)).futureValue + tick() + val offset3 = TimestampOffset(clock.instant(), Map("p5" -> 10L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset3, "p5", 10L)).futureValue + + def env(pid: Pid, seqNr: SeqNr, timestamp: Instant): EventEnvelope[String] = + createEnvelope(pid, seqNr, timestamp, "evt") + + offsetStore.validate(env("p5", 10, offset3.timestamp)).futureValue shouldBe Duplicate + offsetStore.validate(env("p1", 4, offset2.timestamp)).futureValue shouldBe Duplicate + offsetStore.validate(env("p3", 6, offset2.timestamp)).futureValue shouldBe Duplicate + offsetStore.validate(env("p4", 9, offset2.timestamp)).futureValue shouldBe Duplicate + + offsetStore.validate(env("p1", 3, offset1.timestamp)).futureValue shouldBe Duplicate + offsetStore.validate(env("p2", 1, offset1.timestamp)).futureValue shouldBe Duplicate + offsetStore.validate(env("p3", 5, offset1.timestamp)).futureValue shouldBe Duplicate + + offsetStore.validate(env("p1", 2, offset1.timestamp.minusMillis(1))).futureValue shouldBe Duplicate + offsetStore.validate(env("p5", 9, offset3.timestamp.minusMillis(1))).futureValue shouldBe Duplicate + + offsetStore.validate(env("p5", 11, offset3.timestamp)).futureValue shouldNot be(Duplicate) + offsetStore.validate(env("p5", 12, offset3.timestamp.plusMillis(1))).futureValue shouldNot be(Duplicate) + + offsetStore.validate(env("p6", 1, offset3.timestamp.plusMillis(2))).futureValue shouldNot be(Duplicate) + offsetStore.validate(env("p7", 1, offset3.timestamp.minusMillis(1))).futureValue shouldNot be(Duplicate) + } + + "accept known sequence numbers and reject unknown" in { + val projectionId = genRandomProjectionId() + val eventTimestampQueryClock = TestClock.nowMicros() + val offsetStore = createOffsetStore(projectionId, eventTimestampQueryClock = eventTimestampQueryClock) + + val startTime = TestClock.nowMicros().instant() + val offset1 = TimestampOffset(startTime, Map("p1" -> 3L, "p2" -> 1L, "p3" -> 5L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p2", 1L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p3", 5L)).futureValue + + // seqNr 1 is always accepted + val env1 = createEnvelope("p4", 1L, startTime.plusMillis(1), "e4-1") + offsetStore.validate(env1).futureValue shouldBe Accepted + offsetStore.validate(backtrackingEnvelope(env1)).futureValue shouldBe Accepted + // but not if already inflight, seqNr 1 was accepted + offsetStore.addInflight(env1) + val env1Later = createEnvelope("p4", 1L, startTime.plusMillis(1), "e4-1") + offsetStore.validate(env1Later).futureValue shouldBe Duplicate + offsetStore.validate(backtrackingEnvelope(env1Later)).futureValue shouldBe Duplicate + // subsequent seqNr is accepted + val env2 = createEnvelope("p4", 2L, startTime.plusMillis(2), "e4-2") + offsetStore.validate(env2).futureValue shouldBe Accepted + offsetStore.validate(backtrackingEnvelope(env2)).futureValue shouldBe Accepted + offsetStore.addInflight(env2) + // but not when gap + val envP4SeqNr4 = createEnvelope("p4", 4L, startTime.plusMillis(3), "e4-4") + offsetStore.validate(envP4SeqNr4).futureValue shouldBe RejectedSeqNr + // hard reject when gap from backtracking + offsetStore.validate(backtrackingEnvelope(envP4SeqNr4)).futureValue shouldBe RejectedBacktrackingSeqNr + // reject filtered event when gap + offsetStore.validate(filteredEnvelope(envP4SeqNr4)).futureValue shouldBe RejectedSeqNr + // hard reject when filtered event with gap from backtracking + offsetStore + .validate(backtrackingEnvelope(filteredEnvelope(envP4SeqNr4))) + .futureValue shouldBe RejectedBacktrackingSeqNr + // and not if later already inflight, seqNr 2 was accepted + offsetStore.validate(createEnvelope("p4", 1L, startTime.plusMillis(1), "e4-1")).futureValue shouldBe Duplicate + + // +1 to known is accepted + val env3 = createEnvelope("p1", 4L, startTime.plusMillis(4), "e1-4") + offsetStore.validate(env3).futureValue shouldBe Accepted + // but not same + offsetStore.validate(createEnvelope("p3", 5L, startTime, "e3-5")).futureValue shouldBe Duplicate + // but not same, even if it's 1 + offsetStore.validate(createEnvelope("p2", 1L, startTime, "e2-1")).futureValue shouldBe Duplicate + // and not less + offsetStore.validate(createEnvelope("p3", 4L, startTime, "e3-4")).futureValue shouldBe Duplicate + offsetStore.addInflight(env3) + // and then it's not accepted again + offsetStore.validate(env3).futureValue shouldBe Duplicate + offsetStore.validate(backtrackingEnvelope(env3)).futureValue shouldBe Duplicate + // and not when later seqNr is inflight + offsetStore.validate(env2).futureValue shouldBe Duplicate + offsetStore.validate(backtrackingEnvelope(env2)).futureValue shouldBe Duplicate + + // +1 to known, and then also subsequent are accepted (needed for grouped) + val env4 = createEnvelope("p3", 6L, startTime.plusMillis(5), "e3-6") + offsetStore.validate(env4).futureValue shouldBe Accepted + offsetStore.addInflight(env4) + val env5 = createEnvelope("p3", 7L, startTime.plusMillis(6), "e3-7") + offsetStore.validate(env5).futureValue shouldBe Accepted + offsetStore.addInflight(env5) + val env6 = createEnvelope("p3", 8L, startTime.plusMillis(7), "e3-8") + offsetStore.validate(env6).futureValue shouldBe Accepted + offsetStore.addInflight(env6) + + // reject unknown + val env7 = createEnvelope("p5", 7L, startTime.plusMillis(8), "e5-7") + offsetStore.validate(env7).futureValue shouldBe RejectedSeqNr + offsetStore.validate(backtrackingEnvelope(env7)).futureValue shouldBe RejectedBacktrackingSeqNr + // but ok when previous is old + eventTimestampQueryClock.setInstant(startTime.minusSeconds(3600)) + val env8 = createEnvelope("p5", 7L, startTime.plusMillis(5), "e5-7") + offsetStore.validate(env8).futureValue shouldBe Accepted + eventTimestampQueryClock.setInstant(startTime) + offsetStore.addInflight(env8) + // and subsequent seqNr is accepted + val env9 = createEnvelope("p5", 8L, startTime.plusMillis(9), "e5-8") + offsetStore.validate(env9).futureValue shouldBe Accepted + offsetStore.addInflight(env9) + + // reject unknown filtered + val env10 = filteredEnvelope(createEnvelope("p6", 7L, startTime.plusMillis(10), "e6-7")) + offsetStore.validate(env10).futureValue shouldBe RejectedSeqNr + // hard reject when unknown from backtracking + offsetStore.validate(backtrackingEnvelope(env10)).futureValue shouldBe RejectedBacktrackingSeqNr + // hard reject when unknown filtered event from backtracking + offsetStore + .validate(backtrackingEnvelope(filteredEnvelope(env10))) + .futureValue shouldBe RejectedBacktrackingSeqNr + + // it's keeping the inflight that are not in the "stored" state + offsetStore.getInflight() shouldBe Map("p1" -> 4L, "p3" -> 8L, "p4" -> 2L, "p5" -> 8L) + // and they are removed from inflight once they have been stored + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(2), Map("p4" -> 2L)), "p4", 2L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(9), Map("p5" -> 8L)), "p5", 8L)) + .futureValue + offsetStore.getInflight() shouldBe Map("p1" -> 4L, "p3" -> 8L) + } + + "update inflight on error and re-accept element" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + val startTime = TestClock.nowMicros().instant() + + val envelope1 = createEnvelope("p1", 1L, startTime.plusMillis(1), "e1-1") + val envelope2 = createEnvelope("p1", 2L, startTime.plusMillis(2), "e1-2") + val envelope3 = createEnvelope("p1", 3L, startTime.plusMillis(2), "e1-2") + + // seqNr 1 is always accepted + offsetStore.validate(envelope1).futureValue shouldBe Accepted + offsetStore.addInflight(envelope1) + offsetStore.getInflight() shouldBe Map("p1" -> 1L) + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(1), Map("p1" -> 1L)), "p1", 1L)) + .futureValue + offsetStore.getInflight() shouldBe empty + + // seqNr 2 is accepts since it follows seqNr 1 that is stored in state + offsetStore.validate(envelope2).futureValue shouldBe Accepted + // simulate envelope processing error by not adding envelope2 to inflight + + // seqNr 3 is not accepted, still waiting for seqNr 2 + offsetStore.validate(envelope3).futureValue shouldBe RejectedSeqNr + + // offer seqNr 2 once again + offsetStore.validate(envelope2).futureValue shouldBe Accepted + offsetStore.addInflight(envelope2) + offsetStore.getInflight() shouldBe Map("p1" -> 2L) + + // offer seqNr 3 once more + offsetStore.validate(envelope3).futureValue shouldBe Accepted + offsetStore.addInflight(envelope3) + offsetStore.getInflight() shouldBe Map("p1" -> 3L) + + // and they are removed from inflight once they have been stored + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(2), Map("p1" -> 3L)), "p1", 3L)) + .futureValue + offsetStore.getInflight() shouldBe empty + } + + "mapIsAccepted" in { + val projectionId = genRandomProjectionId() + val startTime = TestClock.nowMicros().instant() + val offsetStore = createOffsetStore(projectionId) + + val offset1 = TimestampOffset(startTime, Map("p1" -> 3L, "p2" -> 1L, "p3" -> 5L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p2", 1L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p3", 5L)).futureValue + + // seqNr 1 is always accepted + val env1 = createEnvelope("p4", 1L, startTime.plusMillis(1), "e4-1") + // subsequent seqNr is accepted + val env2 = createEnvelope("p4", 2L, startTime.plusMillis(2), "e4-2") + // but not when gap + val env3 = createEnvelope("p4", 4L, startTime.plusMillis(3), "e4-4") + // ok when previous is known + val env4 = createEnvelope("p1", 4L, startTime.plusMillis(5), "e1-4") + // but not when previous is unknown + val env5 = createEnvelope("p3", 7L, startTime.plusMillis(5), "e3-7") + + offsetStore.validateAll(List(env1, env2, env3, env4, env5)).futureValue shouldBe List( + env1 -> Accepted, + env2 -> Accepted, + env3 -> RejectedSeqNr, + env4 -> Accepted, + env5 -> RejectedSeqNr) + + } + + "accept new revisions for durable state" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + val startTime = TestClock.nowMicros().instant() + val offset1 = TimestampOffset(startTime, Map("p1" -> 3L, "p2" -> 1L, "p3" -> 5L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p2", 1L)).futureValue + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p3", 5L)).futureValue + + // seqNr 1 is always accepted + val env1 = createUpdatedDurableState("p4", 1L, startTime.plusMillis(1), "s4-1") + offsetStore.validate(env1).futureValue shouldBe Accepted + // but not if already inflight, seqNr 1 was accepted + offsetStore.addInflight(env1) + offsetStore + .validate(createUpdatedDurableState("p4", 1L, startTime.plusMillis(1), "s4-1")) + .futureValue shouldBe Duplicate + // subsequent seqNr is accepted + val env2 = createUpdatedDurableState("p4", 2L, startTime.plusMillis(2), "s4-2") + offsetStore.validate(env2).futureValue shouldBe Accepted + offsetStore.addInflight(env2) + // and also ok with gap + offsetStore + .validate(createUpdatedDurableState("p4", 4L, startTime.plusMillis(3), "s4-4")) + .futureValue shouldBe Accepted + // and not if later already inflight, seqNr 2 was accepted + offsetStore + .validate(createUpdatedDurableState("p4", 1L, startTime.plusMillis(1), "s4-1")) + .futureValue shouldBe Duplicate + + // greater than known is accepted + val env3 = createUpdatedDurableState("p1", 4L, startTime.plusMillis(4), "s1-4") + offsetStore.validate(env3).futureValue shouldBe Accepted + // but not same + offsetStore.validate(createUpdatedDurableState("p3", 5L, startTime, "s3-5")).futureValue shouldBe Duplicate + // but not same, even if it's 1 + offsetStore.validate(createUpdatedDurableState("p2", 1L, startTime, "s2-1")).futureValue shouldBe Duplicate + // and not less + offsetStore.validate(createUpdatedDurableState("p3", 4L, startTime, "s3-4")).futureValue shouldBe Duplicate + offsetStore.addInflight(env3) + + // greater than known, and then also subsequent are accepted (needed for grouped) + val env4 = createUpdatedDurableState("p3", 8L, startTime.plusMillis(5), "s3-6") + offsetStore.validate(env4).futureValue shouldBe Accepted + offsetStore.addInflight(env4) + val env5 = createUpdatedDurableState("p3", 9L, startTime.plusMillis(6), "s3-7") + offsetStore.validate(env5).futureValue shouldBe Accepted + offsetStore.addInflight(env5) + val env6 = createUpdatedDurableState("p3", 20L, startTime.plusMillis(7), "s3-8") + offsetStore.validate(env6).futureValue shouldBe Accepted + offsetStore.addInflight(env6) + + // accept unknown + val env7 = createUpdatedDurableState("p5", 7L, startTime.plusMillis(8), "s5-7") + offsetStore.validate(env7).futureValue shouldBe Accepted + offsetStore.addInflight(env7) + + // it's keeping the inflight that are not in the "stored" state + offsetStore.getInflight() shouldBe Map("p1" -> 4L, "p3" -> 20L, "p4" -> 2L, "p5" -> 7L) + // and they are removed from inflight once they have been stored + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(2), Map("p4" -> 2L)), "p4", 2L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plusMillis(9), Map("p5" -> 8L)), "p5", 8L)) + .futureValue + offsetStore.getInflight() shouldBe Map("p1" -> 4L, "p3" -> 20L) + } + + "evict old records from same slice" in { + val projectionId = genRandomProjectionId() + val evictSettings = settings.withTimeWindow(JDuration.ofSeconds(100)).withEvictInterval(JDuration.ofSeconds(10)) + import evictSettings._ + val offsetStore = createOffsetStore(projectionId, evictSettings) + + val startTime = TestClock.nowMicros().instant() + log.debug("Start time [{}]", startTime) + + // these pids have the same slice 645 + val p1 = "p500" + val p2 = "p621" + val p3 = "p742" + val p4 = "p863" + val p5 = "p984" + val p6 = "p3080" + val p7 = "p4290" + val p8 = "p20180" + + offsetStore.saveOffset(OffsetPidSeqNr(TimestampOffset(startTime, Map(p1 -> 1L)), p1, 1L)).futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(JDuration.ofSeconds(1)), Map(p2 -> 1L)), p2, 1L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(JDuration.ofSeconds(2)), Map(p3 -> 1L)), p3, 1L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(evictInterval), Map(p4 -> 1L)), p4, 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(1)), Map(p4 -> 1L)), + p4, + 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(2)), Map(p5 -> 1L)), + p5, + 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(3)), Map(p6 -> 1L)), + p6, + 3L)) + .futureValue + offsetStore.getState().size shouldBe 6 + + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(timeWindow.minusSeconds(10)), Map(p7 -> 1L)), p7, 1L)) + .futureValue + offsetStore.getState().size shouldBe 7 // nothing evicted yet + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).minusSeconds(3)), Map(p8 -> 1L)), + p8, + 1L)) + .futureValue + offsetStore.getState().size shouldBe 8 // still nothing evicted yet + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).plusSeconds(1)), Map(p8 -> 2L)), + p8, + 2L)) + .futureValue + offsetStore.getState().byPid.keySet shouldBe Set(p5, p6, p7, p8) + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).plusSeconds(20)), Map(p8 -> 3L)), + p8, + 3L)) + .futureValue + offsetStore.getState().byPid.keySet shouldBe Set(p7, p8) + } + + "evict old records from different slices" in { + val projectionId = genRandomProjectionId() + val evictSettings = settings.withTimeWindow(JDuration.ofSeconds(100)).withEvictInterval(JDuration.ofSeconds(10)) + import evictSettings._ + val offsetStore = createOffsetStore(projectionId, evictSettings) + + val startTime = TestClock.nowMicros().instant() + log.debug("Start time [{}]", startTime) + + val p1 = "p500" // slice 645 + val p2 = "p92" // slice 905 + val p3 = "p108" // slice 905 + val p4 = "p863" // slice 645 + val p5 = "p984" // slice 645 + val p6 = "p3080" // slice 645 + val p7 = "p4290" // slice 645 + val p8 = "p20180" // slice 645 + + offsetStore.saveOffset(OffsetPidSeqNr(TimestampOffset(startTime, Map(p1 -> 1L)), p1, 1L)).futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(JDuration.ofSeconds(1)), Map(p2 -> 1L)), p2, 1L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(JDuration.ofSeconds(2)), Map(p3 -> 1L)), p3, 1L)) + .futureValue + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(evictInterval), Map(p4 -> 1L)), p4, 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(1)), Map(p4 -> 1L)), + p4, + 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(2)), Map(p5 -> 1L)), + p5, + 1L)) + .futureValue + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(evictInterval).plus(JDuration.ofSeconds(3)), Map(p6 -> 1L)), + p6, + 1L)) + .futureValue + offsetStore.getState().size shouldBe 6 + + offsetStore + .saveOffset(OffsetPidSeqNr(TimestampOffset(startTime.plus(timeWindow.minusSeconds(10)), Map(p7 -> 1L)), p7, 1L)) + .futureValue + offsetStore.getState().size shouldBe 7 // nothing evicted yet + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).minusSeconds(3)), Map(p8 -> 1L)), + p8, + 1L)) + .futureValue + offsetStore.getState().size shouldBe 8 // still nothing evicted yet + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).plusSeconds(1)), Map(p8 -> 2L)), + p8, + 2L)) + .futureValue + offsetStore.getState().byPid.keySet shouldBe Set(p5, p6, p7, p8) + + offsetStore + .saveOffset( + OffsetPidSeqNr( + TimestampOffset(startTime.plus(timeWindow.plus(evictInterval).plusSeconds(20)), Map(p8 -> 3L)), + p8, + 3L)) + .futureValue + offsetStore.getState().byPid.keySet shouldBe Set(p7, p8) + } + + "start from slice offset" in { + val projectionId1 = ProjectionId(UUID.randomUUID().toString, "512-767") + val projectionId2 = ProjectionId(projectionId1.name, "768-1023") + val projectionId3 = ProjectionId(projectionId1.name, "512-1023") + val offsetStore1 = new DynamoDBOffsetStore( + projectionId1, + Some(new TestTimestampSourceProvider(512, 767, clock)), + system, + settings, + client) + val offsetStore2 = new DynamoDBOffsetStore( + projectionId2, + Some(new TestTimestampSourceProvider(768, 1023, clock)), + system, + settings, + client) + + val p1 = "p500" // slice 645, projection1 + val p2 = "p863" // slice 645, projection1 + val p3 = "p11" // slice 656, projection1 + val p4 = "p92" // slice 905, projection2 + + val time1 = TestClock.nowMicros().instant() + val time2 = time1.plusSeconds(1) + val time3 = time1.plusSeconds(2) + val time4 = time1.plusSeconds(3 * 60) // far ahead + + offsetStore1.saveOffset(OffsetPidSeqNr(TimestampOffset(time1, Map(p1 -> 1L)), p1, 1L)).futureValue + offsetStore1.saveOffset(OffsetPidSeqNr(TimestampOffset(time2, Map(p2 -> 1L)), p2, 1L)).futureValue + offsetStore1.saveOffset(OffsetPidSeqNr(TimestampOffset(time3, Map(p3 -> 1L)), p3, 1L)).futureValue + offsetStore2 + .saveOffset(OffsetPidSeqNr(TimestampOffset(time4, Map(p4 -> 1L)), p4, 1L)) + .futureValue + + // after downscaling + val offsetStore3 = new DynamoDBOffsetStore( + projectionId3, + Some(new TestTimestampSourceProvider(512, 1023, clock)), + system, + settings, + client) + + val offsetBySlice = offsetStore3 + .readOffset[TimestampOffsetBySlice]() // this will load from database + .futureValue + .get + .offsets + + offsetBySlice.size shouldBe 3 + offsetBySlice(slice(p1)).timestamp shouldBe time2 + offsetBySlice(slice(p2)).timestamp shouldBe time2 + offsetBySlice(slice(p3)).timestamp shouldBe time3 + offsetBySlice(slice(p4)).timestamp shouldBe time4 + + // getOffset is used by management api, and that should not be adjusted + TimestampOffset.toTimestampOffset(offsetStore3.getOffset().futureValue.get).timestamp shouldBe time4 + } + + "read and save paused" in { + val projectionId = genRandomProjectionId() + val offsetStore = createOffsetStore(projectionId) + + offsetStore.readManagementState().futureValue shouldBe None + + offsetStore.savePaused(paused = true).futureValue + offsetStore.readManagementState().futureValue shouldBe Some(ManagementState(paused = true)) + + offsetStore.savePaused(paused = false).futureValue + offsetStore.readManagementState().futureValue shouldBe Some(ManagementState(paused = false)) + + val offset1 = TimestampOffset(clock.instant(), Map("p1" -> 3L)) + offsetStore.saveOffset(OffsetPidSeqNr(offset1, "p1", 3L)).futureValue + offsetStore.savePaused(paused = true).futureValue + val readOffset1 = offsetStore.readOffset[TimestampOffsetBySlice]().futureValue + readOffset1.get.offsets(slice("p1")) shouldBe offset1 + offsetStore.readManagementState().futureValue shouldBe Some(ManagementState(paused = true)) + } + + // FIXME more tests, see r2dbc + // "set offset" in { + // "clear offset" in { + + } +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/EventSourcedPubSubSpec.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/EventSourcedPubSubSpec.scala new file mode 100644 index 000000000..4339ae389 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/EventSourcedPubSubSpec.scala @@ -0,0 +1,236 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.util.UUID + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.concurrent.duration._ + +import akka.Done +import akka.actor.testkit.typed.scaladsl.LogCapturing +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.actor.testkit.typed.scaladsl.TestProbe +import akka.actor.typed.ActorRef +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.query.scaladsl.DynamoDBReadJournal +import akka.persistence.query.TimestampOffset +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.typed.PersistenceId +import akka.projection.ProjectionBehavior +import akka.projection.ProjectionId +import akka.projection.dynamodb.TestActors.Persister +import akka.projection.dynamodb.scaladsl.DynamoDBProjection +import akka.projection.eventsourced.scaladsl.EventSourcedProvider +import akka.projection.scaladsl.Handler +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import org.scalatest.wordspec.AnyWordSpecLike +import org.slf4j.LoggerFactory + +object EventSourcedPubSubSpec { + + val config: Config = ConfigFactory + .parseString(""" + akka.persistence.dynamodb { + query { + refresh-interval = 3 seconds + # simulate lost messages by overflowing the buffer + buffer-size = 10 + + backtracking { + behind-current-time = 5 seconds + window = 20 seconds + } + } + } + """) + .withFallback(TestConfig.config) + + final case class Processed(projectionId: ProjectionId, envelope: EventEnvelope[String]) + + class TestHandler( + projectionId: ProjectionId, + probe: ActorRef[Processed], + whenDone: EventEnvelope[String] => Future[Done])(implicit ec: ExecutionContext) + extends Handler[EventEnvelope[String]] { + private val log = LoggerFactory.getLogger(getClass) + + override def process(envelope: EventEnvelope[String]): Future[Done] = { + whenDone(envelope).map { _ => + val timestampOffset = envelope.offset.asInstanceOf[TimestampOffset] + val directReplication = timestampOffset.timestamp == timestampOffset.readTimestamp + log.debug( + "{} Processed {}, pid {}, seqNr {}, direct {}", + projectionId.key, + envelope.event, + envelope.persistenceId, + envelope.sequenceNr, + directReplication) + probe ! Processed(projectionId, envelope) + Done + } + } + } + +} + +class EventSourcedPubSubSpec + extends ScalaTestWithActorTestKit(EventSourcedPubSubSpec.config) + with AnyWordSpecLike + with TestDbLifecycle + with TestData + with LogCapturing { + import EventSourcedPubSubSpec._ + + override def typedSystem: ActorSystem[_] = system + private implicit val ec: ExecutionContext = system.executionContext + + private val log = LoggerFactory.getLogger(getClass) + + private val projectionSettings = DynamoDBProjectionSettings(system) + + override protected def beforeAll(): Unit = { + super.beforeAll() + } + + private def startProjections( + entityType: String, + projectionName: String, + nrOfProjections: Int, + processedProbe: ActorRef[Processed], + whenDone: EventEnvelope[String] => Future[Done]): Vector[ActorRef[ProjectionBehavior.Command]] = { + val sliceRanges = EventSourcedProvider.sliceRanges(system, DynamoDBReadJournal.Identifier, nrOfProjections) + + sliceRanges.map { range => + val projectionId = ProjectionId(projectionName, s"${range.min}-${range.max}") + val sourceProvider = + EventSourcedProvider + .eventsBySlices[String](system, DynamoDBReadJournal.Identifier, entityType, range.min, range.max) + val projection = DynamoDBProjection + .atLeastOnce( + projectionId, + Some(projectionSettings), + sourceProvider = sourceProvider, + handler = () => new TestHandler(projectionId, processedProbe.ref, whenDone)) + spawn(ProjectionBehavior(projection)) + }.toVector + } + + private def mkEvent(n: Int): String = f"e$n%05d" + + def expectProcessed(processedProbe: TestProbe[Processed], expectedFrom: Int, expectedTo: Int): Vector[Processed] = { + val numberOfEvents = expectedTo - expectedFrom + 1 + var processed = Vector.empty[Processed] + val expectedEvents = (1 to numberOfEvents).map(mkEvent).toVector + (1 to numberOfEvents).foreach { _ => + // not using receiveMessages(expectedEvents) for better logging in case of failure + try { + processed :+= processedProbe.receiveMessage(25.seconds) + } catch { + case e: AssertionError => + val missing = expectedEvents.diff(processed.map(_.envelope.event)) + log.error(s"Processed [${processed.size}] events, but expected [$numberOfEvents]. " + + s"Missing [${missing.mkString(",")}]. " + + s"Received [${processed.map(p => s"(${p.envelope.event}, ${p.envelope.persistenceId}, ${p.envelope.sequenceNr})").mkString(", ")}]. ") + throw e + } + } + processed + } + + s"A DynamoDB projection with eventsBySlices source and publish-events" must { + + "handle all events without duplicates" in { + val numberOfEntities = 20 + val numberOfEvents = numberOfEntities * 10 + val entityType = nextEntityType() + + val entities = (0 until numberOfEntities).map { n => + val persistenceId = PersistenceId(entityType, s"p$n") + spawn(Persister(persistenceId), s"p$n") + } + + // write some before starting the projections + (1 to 10).foreach { n => + val p = n % numberOfEntities + entities(p) ! Persister.Persist(mkEvent(n)) + } + + val projectionName = UUID.randomUUID().toString + val processedProbe = createTestProbe[Processed]() + val slowEvents = Set(mkEvent(31), mkEvent(32), mkEvent(33)) + val whenDone: EventEnvelope[String] => Future[Done] = { env => + if (slowEvents.contains(env.event)) + akka.pattern.after(500.millis)(Future.successful(Done)) + else + Future.successful(Done) + } + val projections = startProjections(entityType, projectionName, nrOfProjections = 4, processedProbe.ref, whenDone) + + // give them some time to start before writing more events, but it should work anyway + Thread.sleep(500) + + (11 to 20).foreach { n => + val p = n % numberOfEntities + entities(p) ! Persister.Persist(mkEvent(n)) + } + + var processed = Vector.empty[Processed] + processed ++= expectProcessed(processedProbe, 1, 20) + + (21 to 30).foreach { n => + val p = n % numberOfEntities + entities(p) ! Persister.Persist(mkEvent(n)) + } + processed ++= expectProcessed(processedProbe, 21, 30) + + // Processing of 31 is slow in the handler, see whenDone above. + // This will overflow the buffer for the subscribers, simulating lost messages, + // but they should be picked by the queries. + (31 to numberOfEvents - 10).foreach { n => + val p = n % numberOfEntities + entities(p) ! Persister.Persist(mkEvent(n)) + } + processed ++= expectProcessed(processedProbe, 31, numberOfEvents - 10) + + (numberOfEvents - 10 + 1 to numberOfEvents).foreach { n => + val p = n % numberOfEntities + entities(p) ! Persister.Persist(mkEvent(n)) + } + + processed ++= expectProcessed(processedProbe, numberOfEvents - 10 + 1, numberOfEvents) + + val byPid = processed.groupBy(_.envelope.persistenceId) + byPid.foreach { + case (pid, processedByPid) => + // all events of a pid must be processed by the same projection instance + processedByPid.map(_.projectionId).toSet.size shouldBe 1 + // processed events in right order + processedByPid.map(_.envelope.sequenceNr) shouldBe (1 to processedByPid.size).toVector + + val viaPubSub = + processedByPid.filter( + p => + p.envelope.offset.asInstanceOf[TimestampOffset].timestamp == p.envelope.offset + .asInstanceOf[TimestampOffset] + .readTimestamp) + log.info("via pub-sub {}: {}", pid, viaPubSub.map(_.envelope.sequenceNr).mkString(", ")) + } + + val countViaPubSub = processed.count( + p => + p.envelope.offset.asInstanceOf[TimestampOffset].timestamp == p.envelope.offset + .asInstanceOf[TimestampOffset] + .readTimestamp) + log.info("Total via pub-sub: {}", countViaPubSub) + countViaPubSub shouldBe >(0) + + projections.foreach(_ ! ProjectionBehavior.Stop) + } + } + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestActors.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestActors.scala new file mode 100644 index 000000000..ca8dbc833 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestActors.scala @@ -0,0 +1,116 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import akka.Done +import akka.actor.typed.ActorRef +import akka.actor.typed.Behavior +import akka.actor.typed.scaladsl.ActorContext +import akka.actor.typed.scaladsl.Behaviors +import akka.persistence.dynamodb.query.scaladsl.DynamoDBReadJournal +import akka.persistence.typed.PersistenceId +import akka.persistence.typed.ReplicaId +import akka.persistence.typed.ReplicationId +import akka.persistence.typed.scaladsl.EventSourcedBehavior +import akka.persistence.typed.scaladsl.ReplicatedEventSourcing + +object TestActors { + object Persister { + + import akka.persistence.typed.scaladsl.Effect + + sealed trait Command + final case class Persist(payload: Any) extends Command + final case class PersistWithAck(payload: Any, replyTo: ActorRef[Done]) extends Command + final case class PersistAll(payloads: List[Any]) extends Command + final case class Ping(replyTo: ActorRef[Done]) extends Command + final case class GetState(replyTo: ActorRef[String]) extends Command + final case class GetSeqNr(replyTo: ActorRef[Long]) extends Command + final case class Stop(replyTo: ActorRef[Done]) extends Command + + def apply(pid: String): Behavior[Command] = + apply(PersistenceId.ofUniqueId(pid)) + + def apply(pid: PersistenceId): Behavior[Command] = { + apply(pid, "", "", Set.empty) + } + + def apply(pid: PersistenceId, tags: Set[String]): Behavior[Command] = { + apply(pid, "", "", tags) + } + + def apply( + pid: PersistenceId, + journalPluginId: String, + snapshotPluginId: String, + tags: Set[String]): Behavior[Command] = { + Behaviors.setup { context => + eventSourcedBehavior(pid, context) + .withJournalPluginId(journalPluginId) + .withSnapshotPluginId(snapshotPluginId) + .snapshotWhen { + case (_, event, _) => + event.toString.contains("snap") + } + .withTagger(_ => tags) + } + } + + def eventSourcedBehavior( + pid: PersistenceId, + context: ActorContext[Command]): EventSourcedBehavior[Command, Any, String] = { + EventSourcedBehavior[Command, Any, String](persistenceId = pid, "", { + (state, command) => + command match { + case command: Persist => + context.log.debug( + "Persist [{}], pid [{}], seqNr [{}]", + command.payload, + pid.id, + EventSourcedBehavior.lastSequenceNumber(context) + 1) + Effect.persist(command.payload) + case command: PersistWithAck => + context.log.debug( + "Persist [{}], pid [{}], seqNr [{}]", + command.payload, + pid.id, + EventSourcedBehavior.lastSequenceNumber(context) + 1) + Effect.persist(command.payload).thenRun(_ => command.replyTo ! Done) + case command: PersistAll => + if (context.log.isDebugEnabled) + context.log.debug( + "PersistAll [{}], pid [{}], seqNr [{}]", + command.payloads.mkString(","), + pid.id, + EventSourcedBehavior.lastSequenceNumber(context) + 1) + Effect.persist(command.payloads) + case Ping(replyTo) => + replyTo ! Done + Effect.none + case GetState(replyTo) => + replyTo ! state + Effect.none + case GetSeqNr(replyTo) => + replyTo ! EventSourcedBehavior.lastSequenceNumber(context) + Effect.none + case Stop(replyTo) => + replyTo ! Done + Effect.stop() + } + }, (state, event) => if (state == "") event.toString else s"$state|$event") + } + } + + def replicatedEventSourcedPersister(entityType: String, entityId: String): Behavior[Persister.Command] = { + Behaviors.setup[Persister.Command] { context => + ReplicatedEventSourcing.commonJournalConfig( + ReplicationId(entityType, entityId, ReplicaId("dc-1")), + Set(ReplicaId("dc-1")), + DynamoDBReadJournal.Identifier) { _ => + Persister.eventSourcedBehavior(PersistenceId(entityType, entityId), context) + } + } + } +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestClock.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestClock.scala new file mode 100644 index 000000000..3b406a755 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestClock.scala @@ -0,0 +1,52 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.Clock +import java.time.Duration +import java.time.Instant +import java.time.ZoneId +import java.time.ZoneOffset +import java.time.temporal.ChronoUnit +import java.time.temporal.TemporalUnit + +import akka.annotation.InternalApi + +object TestClock { + def nowMillis(): TestClock = new TestClock(ChronoUnit.MILLIS) + def nowMicros(): TestClock = new TestClock(ChronoUnit.MICROS) +} + +/** + * Clock for testing purpose, which is truncated to a resolution (milliseconds or microseconds). + * + * The reason for truncating to the resolution is that Postgres timestamps has the resolution of microseconds but some + * OS/JDK (Linux/JDK17) has Instant resolution of nanoseconds. + */ +@InternalApi private[projection] class TestClock(resolution: TemporalUnit) extends Clock { + + @volatile private var _instant = Instant.now().truncatedTo(resolution) + + override def getZone: ZoneId = ZoneOffset.UTC + + override def withZone(zone: ZoneId): Clock = + throw new UnsupportedOperationException("withZone not supported") + + override def instant(): Instant = + _instant + + def setInstant(newInstant: Instant): Unit = + _instant = newInstant.truncatedTo(resolution) + + /** + * Increase the clock with this duration (truncated to the resolution) + */ + def tick(duration: Duration): Instant = { + val newInstant = _instant.plus(duration).truncatedTo(resolution) + _instant = newInstant + newInstant + } + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestConfig.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestConfig.scala new file mode 100644 index 000000000..c3efda054 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestConfig.scala @@ -0,0 +1,29 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory + +object TestConfig { + lazy val config: Config = { + val defaultConfig = ConfigFactory.load() + + ConfigFactory + .parseString(""" + akka.loglevel = DEBUG + akka.persistence.journal.plugin = "akka.persistence.dynamodb.journal" + akka.persistence.dynamodb { + query { + refresh-interval = 1s + } + client.local.enabled = true + } + akka.actor.testkit.typed.default-timeout = 10s + """) + .withFallback(defaultConfig) + } + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestData.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestData.scala new file mode 100644 index 000000000..bbdd7a353 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestData.scala @@ -0,0 +1,44 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.util.UUID +import java.util.concurrent.atomic.AtomicLong + +import scala.annotation.tailrec + +import akka.actor.typed.ActorSystem +import akka.persistence.Persistence +import akka.persistence.typed.PersistenceId +import akka.projection.ProjectionId + +object TestData { + private val start = 0L // could be something more unique, like currentTimeMillis + private val pidCounter = new AtomicLong(start) +} + +trait TestData { + import TestData.pidCounter + + def typedSystem: ActorSystem[_] + + private lazy val persistenceExt = Persistence(typedSystem) + + def nextPid(): String = s"p-${pidCounter.incrementAndGet()}" + + def nextEntityType(): String = s"TestEntity-${UUID.randomUUID()}" + + def nextPersistenceId(entityType: String): PersistenceId = + PersistenceId.of(entityType, s"${pidCounter.incrementAndGet()}") + + @tailrec final def randomPersistenceIdForSlice(entityType: String, slice: Int): PersistenceId = { + val p = PersistenceId.of(entityType, UUID.randomUUID().toString) + if (persistenceExt.sliceForPersistenceId(p.id) == slice) p + else randomPersistenceIdForSlice(entityType, slice) + } + + def genRandomProjectionId(): ProjectionId = ProjectionId(UUID.randomUUID().toString, "00") + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestDbLifecycle.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestDbLifecycle.scala new file mode 100644 index 000000000..a8fac4e57 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestDbLifecycle.scala @@ -0,0 +1,85 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.jdk.CollectionConverters._ +import scala.jdk.FutureConverters._ +import scala.util.control.NonFatal + +import akka.actor.typed.ActorSystem +import akka.persistence.Persistence +import akka.persistence.dynamodb.DynamoDBSettings +import akka.persistence.dynamodb.util.ClientProvider +import akka.projection.ProjectionId +import akka.projection.dynamodb.scaladsl.CreateTables +import org.scalatest.BeforeAndAfterAll +import org.scalatest.Suite +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient +import software.amazon.awssdk.services.dynamodb.model.AttributeValue +import software.amazon.awssdk.services.dynamodb.model.QueryRequest + +trait TestDbLifecycle extends BeforeAndAfterAll { this: Suite => + + def typedSystem: ActorSystem[_] + + def testConfigPath: String = "akka.projection.dynamodb" + + lazy val settings: DynamoDBProjectionSettings = + DynamoDBProjectionSettings(typedSystem.settings.config.getConfig(testConfigPath)) + + lazy val dynamoDBSettings: DynamoDBSettings = + DynamoDBSettings( + typedSystem.settings.config + .getConfig(settings.useClient.replace(".client", ""))) + + lazy val persistenceExt: Persistence = Persistence(typedSystem) + + lazy val client: DynamoDbAsyncClient = ClientProvider(typedSystem).clientFor(settings.useClient) + + lazy val localDynamoDB: Boolean = ClientProvider(typedSystem).clientSettingsFor(settings.useClient).local.isDefined + + override protected def beforeAll(): Unit = { + if (localDynamoDB) { + try { + Await.result( + akka.persistence.dynamodb.util.scaladsl.CreateTables + .createJournalTable(typedSystem, dynamoDBSettings, client, deleteIfExists = true), + 10.seconds) + Await.result( + CreateTables.createTimestampOffsetStoreTable(typedSystem, settings, client, deleteIfExists = true), + 10.seconds) + } catch { + case NonFatal(ex) => throw new RuntimeException(s"Test db creation failed", ex) + } + } + + super.beforeAll() + } + + // directly get an offset item (timestamp or sequence number) from the timestamp offset table + def getOffsetItemFor( + projectionId: ProjectionId, + slice: Int, + persistenceId: String = "_"): Option[Map[String, AttributeValue]] = { + import akka.projection.dynamodb.internal.OffsetStoreDao.OffsetStoreAttributes._ + + val attributes = + Map( + ":nameSlice" -> AttributeValue.fromS(s"${projectionId.name}-$slice"), + ":pid" -> AttributeValue.fromS(persistenceId)).asJava + + val request = QueryRequest.builder + .tableName(settings.timestampOffsetTable) + .consistentRead(true) + .keyConditionExpression(s"$NameSlice = :nameSlice AND $Pid = :pid") + .expressionAttributeValues(attributes) + .build() + + Await.result(client.query(request).asScala, 10.seconds).items.asScala.headOption.map(_.asScala.toMap) + } + +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestSourceProviderWithInput.scala b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestSourceProviderWithInput.scala new file mode 100644 index 000000000..117bdd879 --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/akka/projection/dynamodb/TestSourceProviderWithInput.scala @@ -0,0 +1,99 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.Instant +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicReference + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.concurrent.Promise +import scala.jdk.CollectionConverters._ + +import akka.NotUsed +import akka.actor.typed.ActorRef +import akka.actor.typed.ActorSystem +import akka.actor.typed.scaladsl.adapter._ +import akka.persistence.Persistence +import akka.persistence.query.TimestampOffset +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.query.typed.scaladsl.EventTimestampQuery +import akka.persistence.query.typed.scaladsl.LoadEventQuery +import akka.projection.BySlicesSourceProvider +import akka.projection.scaladsl.SourceProvider +import akka.stream.OverflowStrategy +import akka.stream.scaladsl.Source + +class TestSourceProviderWithInput()(implicit val system: ActorSystem[_]) + extends SourceProvider[TimestampOffset, EventEnvelope[String]] + with BySlicesSourceProvider + with EventTimestampQuery + with LoadEventQuery { + + private implicit val ec: ExecutionContext = system.executionContext + private val persistenceExt = Persistence(system) + + private val _input = new AtomicReference[Promise[ActorRef[EventEnvelope[String]]]](Promise()) + + def input: Future[ActorRef[EventEnvelope[String]]] = _input.get().future + + private val envelopes = new ConcurrentLinkedQueue[EventEnvelope[String]] + + override def source(offset: () => Future[Option[TimestampOffset]]): Future[Source[EventEnvelope[String], NotUsed]] = { + val oldPromise = _input.get() + _input.set(Promise()) + offset().map { _ => + Source + .actorRef[EventEnvelope[String]]( + PartialFunction.empty, + PartialFunction.empty, + bufferSize = 1024, + OverflowStrategy.fail) + .map { env => + envelopes.offer(env) + env + } + .mapMaterializedValue { ref => + val typedRef = ref.toTyped[EventEnvelope[String]] + oldPromise.trySuccess(typedRef) + _input.get().trySuccess(typedRef) + NotUsed + } + } + } + + override def extractOffset(envelope: EventEnvelope[String]): TimestampOffset = + envelope.offset.asInstanceOf[TimestampOffset] + + override def extractCreationTime(envelope: EventEnvelope[String]): Long = + envelope.timestamp + + override def minSlice: Int = 0 + + override def maxSlice: Int = persistenceExt.numberOfSlices - 1 + + override def timestampOf(persistenceId: String, sequenceNr: Long): Future[Option[Instant]] = { + Future.successful(envelopes.iterator().asScala.collectFirst { + case env + if env.persistenceId == persistenceId && env.sequenceNr == sequenceNr && env.offset + .isInstanceOf[TimestampOffset] => + env.offset.asInstanceOf[TimestampOffset].timestamp + }) + } + + override def loadEnvelope[Event](persistenceId: String, sequenceNr: Long): Future[EventEnvelope[Event]] = { + envelopes.iterator().asScala.collectFirst { + case env if env.persistenceId == persistenceId && env.sequenceNr == sequenceNr => + env.asInstanceOf[EventEnvelope[Event]] + } match { + case Some(env) => Future.successful(env) + case None => + Future.failed( + new NoSuchElementException( + s"Event with persistenceId [$persistenceId] and sequenceNr [$sequenceNr] not found.")) + } + } +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/projection/docs/config/ProjectionTimeToLiveSettingsDocExample.scala b/akka-projection-dynamodb-integration/src/test/scala/projection/docs/config/ProjectionTimeToLiveSettingsDocExample.scala new file mode 100644 index 000000000..62d3f6bfe --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/projection/docs/config/ProjectionTimeToLiveSettingsDocExample.scala @@ -0,0 +1,58 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package projection.docs.config + +import scala.concurrent.duration._ + +import akka.projection.dynamodb.DynamoDBProjectionSettings +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import org.scalatest.matchers.should.Matchers +import org.scalatest.wordspec.AnyWordSpec + +object ProjectionTimeToLiveSettingsDocExample { + + val ttlConfig: Config = ConfigFactory.load(ConfigFactory.parseString(""" + //#time-to-live + akka.projection.dynamodb.time-to-live { + projections { + "some-projection" { + offset-time-to-live = 7 days + } + "projection-*" { + offset-time-to-live = 14 days + } + } + } + //#time-to-live + """)) +} + +class ProjectionTimeToLiveSettingsDocExample extends AnyWordSpec with Matchers { + import ProjectionTimeToLiveSettingsDocExample._ + + def dynamoDBProjectionSettings(config: Config): DynamoDBProjectionSettings = + DynamoDBProjectionSettings(config.getConfig("akka.projection.dynamodb")) + + "Projection Time to Live (TTL) docs" should { + + "have example of setting offset-time-to-live" in { + val settings = dynamoDBProjectionSettings(ttlConfig) + + val someTtlSettings = settings.timeToLiveSettings.projections.get("some-projection") + someTtlSettings.offsetTimeToLive shouldBe Some(7.days) + + val ttlSettings1 = settings.timeToLiveSettings.projections.get("projection-1") + ttlSettings1.offsetTimeToLive shouldBe Some(14.days) + + val ttlSettings2 = settings.timeToLiveSettings.projections.get("projection-2") + ttlSettings2.offsetTimeToLive shouldBe Some(14.days) + + val defaultTtlSettings = settings.timeToLiveSettings.projections.get("other-projection") + defaultTtlSettings.offsetTimeToLive shouldBe None + } + + } +} diff --git a/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala b/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala new file mode 100644 index 000000000..857bf33dc --- /dev/null +++ b/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala @@ -0,0 +1,467 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package projection.docs.scaladsl + +import java.time.Instant + +import scala.concurrent.Await +import scala.concurrent.ExecutionContext +import scala.concurrent.duration._ + +import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit +import akka.actor.typed.ActorSystem +import akka.cluster.sharding.typed.scaladsl.EntityTypeKey +import akka.persistence.query.Offset +import akka.persistence.query.typed.EventEnvelope +import akka.projection.scaladsl.SourceProvider +import akka.serialization.jackson.CborSerializable +import com.typesafe.config.Config +import com.typesafe.config.ConfigFactory +import org.scalatest.concurrent.Futures +import org.scalatest.wordspec.AnyWordSpecLike +import org.slf4j.LoggerFactory +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient + +object ProjectionDocExample { + val config: Config = ConfigFactory.load(ConfigFactory.parseString(""" + //#local-mode + akka.persistence.dynamodb { + client.local.enabled = true + } + //#local-mode + """)) + + object ShoppingCart { + val EntityKey: EntityTypeKey[Command] = EntityTypeKey[Command]("ShoppingCart") + + sealed trait Command extends CborSerializable + + sealed trait Event extends CborSerializable { + def cartId: String + } + + final case class ItemAdded(cartId: String, itemId: String, quantity: Int) extends Event + final case class ItemRemoved(cartId: String, itemId: String) extends Event + final case class ItemQuantityAdjusted(cartId: String, itemId: String, newQuantity: Int) extends Event + final case class CheckedOut(cartId: String, eventTime: Instant) extends Event + } + + object HandlerExample { + //#handler + import akka.Done + import akka.persistence.query.typed.EventEnvelope + import akka.projection.scaladsl.Handler + import software.amazon.awssdk.services.dynamodb.model.AttributeValue + import software.amazon.awssdk.services.dynamodb.model.PutItemRequest + + import scala.concurrent.Future + import scala.jdk.CollectionConverters._ + import scala.jdk.FutureConverters._ + + class ShoppingCartHandler(client: DynamoDbAsyncClient)(implicit ec: ExecutionContext) + extends Handler[EventEnvelope[ShoppingCart.Event]] { + private val logger = LoggerFactory.getLogger(getClass) + + override def process(envelope: EventEnvelope[ShoppingCart.Event]): Future[Done] = { + envelope.event match { + case ShoppingCart.CheckedOut(cartId, time) => + logger.info(s"Shopping cart $cartId was checked out at $time") + + val attributes = Map( + "id" -> AttributeValue.fromS(cartId), + "time" -> AttributeValue.fromN(time.toEpochMilli.toString)).asJava + + client + .putItem( + PutItemRequest.builder + .tableName("orders") + .item(attributes) + .build()) + .asScala + .map(_ => Done) + + case otherEvent => + logger.debug(s"Shopping cart ${otherEvent.cartId} changed by $otherEvent") + Future.successful(Done) + } + } + } + //#handler + } + + object TransactHandlerExample { + //#transact-handler + import akka.persistence.query.typed.EventEnvelope + import akka.projection.dynamodb.scaladsl.DynamoDBTransactHandler + import software.amazon.awssdk.services.dynamodb.model.AttributeValue + import software.amazon.awssdk.services.dynamodb.model.Put + import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + + import scala.concurrent.Future + import scala.jdk.CollectionConverters._ + + class ShoppingCartTransactHandler extends DynamoDBTransactHandler[EventEnvelope[ShoppingCart.Event]] { + private val logger = LoggerFactory.getLogger(getClass) + + override def process(envelope: EventEnvelope[ShoppingCart.Event]): Future[Iterable[TransactWriteItem]] = { + envelope.event match { + case ShoppingCart.CheckedOut(cartId, time) => + logger.info(s"Shopping cart $cartId was checked out at $time") + + val attributes = Map( + "id" -> AttributeValue.fromS(cartId), + "time" -> AttributeValue.fromN(time.toEpochMilli.toString)).asJava + + Future.successful( + Seq( + TransactWriteItem.builder + .put( + Put.builder + .tableName("orders") + .item(attributes) + .build()) + .build())) + + case otherEvent => + logger.debug(s"Shopping cart ${otherEvent.cartId} changed by $otherEvent") + Future.successful(Seq.empty) + } + } + } + //#transact-handler + } + + object GroupedHandlerExample { + //#grouped-handler + import akka.Done + import akka.persistence.query.typed.EventEnvelope + import akka.projection.scaladsl.Handler + import software.amazon.awssdk.services.dynamodb.model.AttributeValue + import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemRequest + import software.amazon.awssdk.services.dynamodb.model.PutRequest + import software.amazon.awssdk.services.dynamodb.model.WriteRequest + + import scala.concurrent.Future + import scala.jdk.CollectionConverters._ + import scala.jdk.FutureConverters._ + + class GroupedShoppingCartHandler(client: DynamoDbAsyncClient)(implicit ec: ExecutionContext) + extends Handler[Seq[EventEnvelope[ShoppingCart.Event]]] { + private val logger = LoggerFactory.getLogger(getClass) + + override def process(envelopes: Seq[EventEnvelope[ShoppingCart.Event]]): Future[Done] = { + val items = envelopes.flatMap { envelope => + envelope.event match { + case ShoppingCart.CheckedOut(cartId, time) => + logger.info(s"Shopping cart $cartId was checked out at $time") + + val attributes = + Map("id" -> AttributeValue.fromS(cartId), "time" -> AttributeValue.fromN(time.toEpochMilli.toString)).asJava + + Some( + WriteRequest.builder + .putRequest( + PutRequest.builder + .item(attributes) + .build()) + .build()) + + case otherEvent => + logger.debug(s"Shopping cart ${otherEvent.cartId} changed by $otherEvent") + None + } + }.asJava + + client + .batchWriteItem( + BatchWriteItemRequest.builder + .requestItems(Map("orders" -> items).asJava) + .build()) + .asScala + .map(_ => Done) + } + } + //#grouped-handler + } + + object GroupedTransactHandlerExample { + //#grouped-transact-handler + import akka.persistence.query.typed.EventEnvelope + import akka.projection.dynamodb.scaladsl.DynamoDBTransactHandler + import software.amazon.awssdk.services.dynamodb.model.AttributeValue + import software.amazon.awssdk.services.dynamodb.model.Put + import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + + import scala.concurrent.Future + import scala.jdk.CollectionConverters._ + + class GroupedShoppingCartTransactHandler extends DynamoDBTransactHandler[Seq[EventEnvelope[ShoppingCart.Event]]] { + private val logger = LoggerFactory.getLogger(getClass) + + override def process(envelopes: Seq[EventEnvelope[ShoppingCart.Event]]): Future[Iterable[TransactWriteItem]] = { + val items = envelopes.flatMap { envelope => + envelope.event match { + case ShoppingCart.CheckedOut(cartId, time) => + logger.info(s"Shopping cart $cartId was checked out at $time") + + val attributes = Map( + "id" -> AttributeValue.fromS(cartId), + "time" -> AttributeValue.fromN(time.toEpochMilli.toString)).asJava + + Seq( + TransactWriteItem.builder + .put( + Put.builder + .tableName("orders") + .item(attributes) + .build()) + .build()) + + case otherEvent => + logger.debug(s"Shopping cart ${otherEvent.cartId} changed by $otherEvent") + Seq.empty + } + } + Future.successful(items) + } + } + //#grouped-transact-handler + } + + def initExample(implicit system: ActorSystem[_]): Unit = { + import TransactHandlerExample.ShoppingCartTransactHandler + + //#init-projections + import akka.cluster.sharding.typed.ShardedDaemonProcessSettings + import akka.cluster.sharding.typed.scaladsl.ShardedDaemonProcess + import akka.persistence.dynamodb.query.scaladsl.DynamoDBReadJournal + import akka.persistence.query.typed.EventEnvelope + import akka.projection.Projection + import akka.projection.ProjectionBehavior + import akka.projection.ProjectionId + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + import akka.projection.eventsourced.scaladsl.EventSourcedProvider + import akka.projection.scaladsl.SourceProvider + + def sourceProvider(sliceRange: Range): SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]] = + EventSourcedProvider.eventsBySlices[ShoppingCart.Event]( + system, + readJournalPluginId = DynamoDBReadJournal.Identifier, + ShoppingCart.EntityKey.name, + sliceRange.min, + sliceRange.max) + + def projection(sliceRange: Range): Projection[EventEnvelope[ShoppingCart.Event]] = { + val minSlice = sliceRange.min + val maxSlice = sliceRange.max + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + DynamoDBProjection.exactlyOnce( + projectionId, + settings = None, + sourceProvider(sliceRange), + handler = () => new ShoppingCartTransactHandler) + } + + ShardedDaemonProcess(system).initWithContext( + name = "ShoppingCartProjection", + initialNumberOfInstances = 4, + behaviorFactory = { daemonContext => + val sliceRanges = + EventSourcedProvider.sliceRanges(system, DynamoDBReadJournal.Identifier, daemonContext.totalProcesses) + val sliceRange = sliceRanges(daemonContext.processNumber) + ProjectionBehavior(projection(sliceRange)) + }, + ShardedDaemonProcessSettings(system), + stopMessage = ProjectionBehavior.Stop) + //#init-projections + } + + def exactlyOnceExample( + sourceProvider: SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]], + minSlice: Int, + maxSlice: Int)(implicit system: ActorSystem[_]): Unit = { + import TransactHandlerExample.ShoppingCartTransactHandler + + //#exactly-once + import akka.projection.ProjectionId + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + val projection = DynamoDBProjection.exactlyOnce( + projectionId, + settings = None, + sourceProvider, + handler = () => new ShoppingCartTransactHandler) + //#exactly-once + + val _ = projection + } + + def atLeastOnceExample( + sourceProvider: akka.projection.scaladsl.SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]], + minSlice: Int, + maxSlice: Int, + client: DynamoDbAsyncClient)(implicit system: ActorSystem[_], ec: ExecutionContext): Unit = { + import HandlerExample.ShoppingCartHandler + + //#at-least-once + import akka.projection.ProjectionId + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + val projection = + DynamoDBProjection + .atLeastOnce(projectionId, settings = None, sourceProvider, handler = () => new ShoppingCartHandler(client)) + .withSaveOffset(afterEnvelopes = 100, afterDuration = 500.millis) + //#at-least-once + + val _ = projection + } + + def exactlyOnceGroupedWithinExample( + sourceProvider: akka.projection.scaladsl.SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]], + minSlice: Int, + maxSlice: Int)(implicit system: ActorSystem[_]): Unit = { + import GroupedTransactHandlerExample.GroupedShoppingCartTransactHandler + + //#exactly-once-grouped-within + import akka.projection.ProjectionId + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + val projection = + DynamoDBProjection + .exactlyOnceGroupedWithin( + projectionId, + settings = None, + sourceProvider, + handler = () => new GroupedShoppingCartTransactHandler) + .withGroup(groupAfterEnvelopes = 20, groupAfterDuration = 500.millis) + //#exactly-once-grouped-within + + val _ = projection + } + + def atLeastOnceGroupedWithinExample( + sourceProvider: akka.projection.scaladsl.SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]], + minSlice: Int, + maxSlice: Int, + client: DynamoDbAsyncClient)(implicit system: ActorSystem[_], ec: ExecutionContext): Unit = { + import GroupedHandlerExample.GroupedShoppingCartHandler + + //#at-least-once-grouped-within + import akka.projection.ProjectionId + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + val projection = + DynamoDBProjection + .atLeastOnceGroupedWithin( + projectionId, + settings = None, + sourceProvider, + handler = () => new GroupedShoppingCartHandler(client)) + .withGroup(groupAfterEnvelopes = 20, groupAfterDuration = 500.millis) + //#at-least-once-grouped-within + + val _ = projection + } + + object MultiPluginExample { + val config = + s""" + // #second-projection-config + second-projection-dynamodb = $${akka.projection.dynamodb} + second-projection-dynamodb { + offset-store { + # specific projection offset store settings here + } + use-client = "second-dynamodb.client" + } + // #second-projection-config + + // #second-projection-config-with-client + second-projection-dynamodb = $${akka.projection.dynamodb} + second-projection-dynamodb { + offset-store { + # specific projection offset store settings here + } + client = $${akka.persistence.dynamodb.client} + client { + # specific client settings for offset store and projection handler here + } + use-client = "second-projection-dynamodb.client" + } + // #second-projection-config-with-client + """ + + def projectionWithSecondPlugin( + sourceProvider: akka.projection.scaladsl.SourceProvider[Offset, EventEnvelope[ShoppingCart.Event]], + minSlice: Int, + maxSlice: Int, + client: DynamoDbAsyncClient)(implicit system: ActorSystem[_], ec: ExecutionContext): Unit = { + import HandlerExample.ShoppingCartHandler + + //#projection-settings + import akka.projection.ProjectionId + import akka.projection.dynamodb.DynamoDBProjectionSettings + import akka.projection.dynamodb.scaladsl.DynamoDBProjection + + val projectionId = ProjectionId("ShoppingCarts", s"carts-$minSlice-$maxSlice") + + val settings = Some(DynamoDBProjectionSettings(system.settings.config.getConfig("second-projection-dynamodb"))) + + val projection = + DynamoDBProjection.atLeastOnce( + projectionId, + settings, + sourceProvider, + handler = () => new ShoppingCartHandler(client)) + //#projection-settings + + val _ = projection + } + } +} + +class ProjectionDocExample + extends ScalaTestWithActorTestKit(ProjectionDocExample.config) + with AnyWordSpecLike + with Futures { + + "Projection docs" should { + + "have example of creating tables locally (Scala)" in { + //#create-tables + import akka.persistence.dynamodb.util.ClientProvider + import akka.projection.dynamodb.DynamoDBProjectionSettings + import akka.projection.dynamodb.scaladsl.CreateTables + import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient + + val dynamoDBConfigPath = "akka.projection.dynamodb" + + val settings: DynamoDBProjectionSettings = + DynamoDBProjectionSettings(system.settings.config.getConfig(dynamoDBConfigPath)) + + val client: DynamoDbAsyncClient = ClientProvider(system).clientFor(settings.useClient) + + // create timestamp offset table, synchronously + Await.result( + CreateTables.createTimestampOffsetStoreTable(system, settings, client, deleteIfExists = true), + 10.seconds) + //#create-tables + } + + "have example of creating tables locally (Java)" in { + projection.docs.javadsl.ProjectionDocExample.createTables(system) + } + + } +} diff --git a/akka-projection-dynamodb/scripts/create-tables.sh b/akka-projection-dynamodb/scripts/create-tables.sh new file mode 100644 index 000000000..92d7d59f6 --- /dev/null +++ b/akka-projection-dynamodb/scripts/create-tables.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +#create-timestamp-offset-table +aws dynamodb create-table \ + --table-name timestamp_offset \ + --attribute-definitions \ + AttributeName=name_slice,AttributeType=S \ + AttributeName=pid,AttributeType=S \ + --key-schema \ + AttributeName=name_slice,KeyType=HASH \ + AttributeName=pid,KeyType=RANGE \ + --provisioned-throughput \ + ReadCapacityUnits=5,WriteCapacityUnits=5 +#create-timestamp-offset-table diff --git a/akka-projection-dynamodb/src/main/resources/reference.conf b/akka-projection-dynamodb/src/main/resources/reference.conf new file mode 100644 index 000000000..03b2db60d --- /dev/null +++ b/akka-projection-dynamodb/src/main/resources/reference.conf @@ -0,0 +1,64 @@ +# This defines the default configuration for akka-projection-dynamodb. +# Make your edits/overrides in your application.conf. + +//#projection-config +akka.projection.dynamodb { + + offset-store { + # the DynamoDB table name for the offset store + timestamp-offset-table = "timestamp_offset" + + # The offset store will keep track of persistence ids and sequence numbers + # within this time window from latest offset. + time-window = 5 minutes + + # Keep this number of entries. Don't evict old entries until this threshold + # has been reached. + keep-number-of-entries = 10000 + + # Remove old entries outside the time-window from the offset store memory + # with this frequency. + evict-interval = 10 seconds + + # Trying to batch insert offsets in batches of this size. + offset-batch-size = 20 + } + + # By default it shares DynamoDB client with akka-persistence-dynamodb (write side). + # To use a separate client for projections this can be + # set to another config path that defines the config based on + # akka.persistence.dynamodb.client config. + use-client = "akka.persistence.dynamodb.client" + + # Filtered events are not actually filtered but passed through the handling flow + # for atLeastOnceFlow, in some applications this is fine, set to false to disable + # the info logged when seeing such filtered events + warn-about-filtered-events-in-flow = true +} +//#projection-config + +//#time-to-live-settings +akka.projection.dynamodb { + # Time to Live (TTL) settings + time-to-live { + projection-defaults { + # Set a time-to-live duration on all offsets when they are updated. + # Disabled when set to `off` or `none`. + offset-time-to-live = off + } + + # Time-to-live settings per projection name. + # See `projection-defaults` for possible settings and default values. + # Prefix matching is supported by using * at the end of a projection name. + projections { + # Example configuration: + # "some-projection" { + # offset-time-to-live = 7 days + # } + # "projection-*" { + # offset-time-to-live = 14 days + # } + } + } +} +//#time-to-live-settings diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/DynamoDBProjectionSettings.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/DynamoDBProjectionSettings.scala new file mode 100644 index 000000000..b9090a73d --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/DynamoDBProjectionSettings.scala @@ -0,0 +1,190 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb + +import java.time.{ Duration => JDuration } + +import scala.concurrent.duration._ +import scala.jdk.CollectionConverters._ +import scala.jdk.DurationConverters._ + +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.ConfigHelpers +import akka.persistence.dynamodb.WildcardMap +import com.typesafe.config.Config +import com.typesafe.config.ConfigObject + +object DynamoDBProjectionSettings { + + val DefaultConfigPath = "akka.projection.dynamodb" + + /** + * Scala API: Load configuration from `akka.projection.dynamodb`. + */ + def apply(system: ActorSystem[_]): DynamoDBProjectionSettings = + apply(system.settings.config.getConfig(DefaultConfigPath)) + + /** + * Java API: Load configuration from `akka.projection.dynamodb`. + */ + def create(system: ActorSystem[_]): DynamoDBProjectionSettings = + apply(system) + + /** + * Scala API: From custom configuration corresponding to `akka.projection.dynamodb`. + */ + def apply(config: Config): DynamoDBProjectionSettings = { + new DynamoDBProjectionSettings( + timestampOffsetTable = config.getString("offset-store.timestamp-offset-table"), + useClient = config.getString("use-client"), + timeWindow = config.getDuration("offset-store.time-window"), + keepNumberOfEntries = config.getInt("offset-store.keep-number-of-entries"), + evictInterval = config.getDuration("offset-store.evict-interval"), + warnAboutFilteredEventsInFlow = config.getBoolean("warn-about-filtered-events-in-flow"), + offsetBatchSize = config.getInt("offset-store.offset-batch-size"), + timeToLiveSettings = TimeToLiveSettings(config.getConfig("time-to-live"))) + } + + /** + * Java API: From custom configuration corresponding to `akka.projection.dynamodb`. + */ + def create(config: Config): DynamoDBProjectionSettings = + apply(config) + +} + +final class DynamoDBProjectionSettings private ( + val timestampOffsetTable: String, + val useClient: String, + val timeWindow: JDuration, + val keepNumberOfEntries: Int, + val evictInterval: JDuration, + val warnAboutFilteredEventsInFlow: Boolean, + val offsetBatchSize: Int, + val timeToLiveSettings: TimeToLiveSettings) { + + def withTimestampOffsetTable(timestampOffsetTable: String): DynamoDBProjectionSettings = + copy(timestampOffsetTable = timestampOffsetTable) + + def withUseClient(clientConfigPath: String): DynamoDBProjectionSettings = + copy(useClient = clientConfigPath) + + def withTimeWindow(timeWindow: FiniteDuration): DynamoDBProjectionSettings = + copy(timeWindow = timeWindow.toJava) + + def withTimeWindow(timeWindow: JDuration): DynamoDBProjectionSettings = + copy(timeWindow = timeWindow) + + def withKeepNumberOfEntries(keepNumberOfEntries: Int): DynamoDBProjectionSettings = + copy(keepNumberOfEntries = keepNumberOfEntries) + + def withEvictInterval(evictInterval: FiniteDuration): DynamoDBProjectionSettings = + copy(evictInterval = evictInterval.toJava) + + def withEvictInterval(evictInterval: JDuration): DynamoDBProjectionSettings = + copy(evictInterval = evictInterval) + + def withWarnAboutFilteredEventsInFlow(warnAboutFilteredEventsInFlow: Boolean): DynamoDBProjectionSettings = + copy(warnAboutFilteredEventsInFlow = warnAboutFilteredEventsInFlow) + + def withOffsetBatchSize(offsetBatchSize: Int): DynamoDBProjectionSettings = + copy(offsetBatchSize = offsetBatchSize) + + def withTimeToLiveSettings(timeToLiveSettings: TimeToLiveSettings): DynamoDBProjectionSettings = + copy(timeToLiveSettings = timeToLiveSettings) + + private def copy( + timestampOffsetTable: String = timestampOffsetTable, + useClient: String = useClient, + timeWindow: JDuration = timeWindow, + keepNumberOfEntries: Int = keepNumberOfEntries, + evictInterval: JDuration = evictInterval, + warnAboutFilteredEventsInFlow: Boolean = warnAboutFilteredEventsInFlow, + offsetBatchSize: Int = offsetBatchSize, + timeToLiveSettings: TimeToLiveSettings = timeToLiveSettings) = + new DynamoDBProjectionSettings( + timestampOffsetTable, + useClient, + timeWindow, + keepNumberOfEntries, + evictInterval, + warnAboutFilteredEventsInFlow, + offsetBatchSize, + timeToLiveSettings) + + override def toString = + s"DynamoDBProjectionSettings($timestampOffsetTable, $useClient, $timeWindow, $keepNumberOfEntries, $evictInterval, $warnAboutFilteredEventsInFlow, $offsetBatchSize)" +} + +object TimeToLiveSettings { + val defaults: TimeToLiveSettings = + new TimeToLiveSettings(projections = WildcardMap(Seq.empty, ProjectionTimeToLiveSettings.defaults)) + + /** + * Scala API: Create from configuration corresponding to `akka.projection.dynamodb.time-to-live`. + */ + def apply(config: Config): TimeToLiveSettings = { + val projections: WildcardMap[ProjectionTimeToLiveSettings] = { + val defaults = config.getConfig("projection-defaults") + val defaultSettings = ProjectionTimeToLiveSettings(defaults) + val entries = config.getConfig("projections").root.entrySet.asScala + val perEntitySettings = entries.toSeq.flatMap { entry => + (entry.getKey, entry.getValue) match { + case (key: String, value: ConfigObject) => + val settings = ProjectionTimeToLiveSettings(value.toConfig.withFallback(defaults)) + Some(key -> settings) + case _ => None + } + } + WildcardMap(perEntitySettings, defaultSettings) + } + new TimeToLiveSettings(projections) + } + + /** + * Java API: Create from configuration corresponding to `akka.projection.dynamodb.time-to-live`. + */ + def create(config: Config): TimeToLiveSettings = apply(config) +} + +final class TimeToLiveSettings private (val projections: WildcardMap[ProjectionTimeToLiveSettings]) { + + def withProjection(name: String, settings: ProjectionTimeToLiveSettings): TimeToLiveSettings = + copy(projections = projections.updated(name, settings)) + + private def copy(projections: WildcardMap[ProjectionTimeToLiveSettings]): TimeToLiveSettings = + new TimeToLiveSettings(projections) +} + +object ProjectionTimeToLiveSettings { + val defaults: ProjectionTimeToLiveSettings = + new ProjectionTimeToLiveSettings(offsetTimeToLive = None) + + /** + * Scala API: Create from configuration corresponding to `akka.projection.dynamodb.time-to-live.projections`. + */ + def apply(config: Config): ProjectionTimeToLiveSettings = + new ProjectionTimeToLiveSettings(offsetTimeToLive = ConfigHelpers.optDuration(config, "offset-time-to-live")) + + /** + * Java API: Create from configuration corresponding to `akka.projection.dynamodb.time-to-live.projections`. + */ + def create(config: Config): ProjectionTimeToLiveSettings = apply(config) +} + +final class ProjectionTimeToLiveSettings private (val offsetTimeToLive: Option[FiniteDuration]) { + + def withOffsetTimeToLive(offsetTimeToLive: FiniteDuration): ProjectionTimeToLiveSettings = + copy(offsetTimeToLive = Some(offsetTimeToLive)) + + def withOffsetTimeToLive(offsetTimeToLive: JDuration): ProjectionTimeToLiveSettings = + copy(offsetTimeToLive = Some(offsetTimeToLive.toScala)) + + def withNoOffsetTimeToLive(): ProjectionTimeToLiveSettings = + copy(offsetTimeToLive = None) + + private def copy(offsetTimeToLive: Option[FiniteDuration]): ProjectionTimeToLiveSettings = + new ProjectionTimeToLiveSettings(offsetTimeToLive) +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBHandlerAdapter.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBHandlerAdapter.scala new file mode 100644 index 000000000..db11741cb --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBHandlerAdapter.scala @@ -0,0 +1,50 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.internal + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.jdk.CollectionConverters._ +import scala.jdk.FutureConverters._ + +import akka.Done +import akka.annotation.InternalApi +import akka.projection.dynamodb.javadsl +import akka.projection.dynamodb.scaladsl +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + +/** + * INTERNAL API: Adapter from javadsl.DynamoDBTransactHandler to scaladsl.DynamoDBTransactHandler + */ +@InternalApi private[projection] class DynamoDBTransactHandlerAdapter[Envelope]( + delegate: javadsl.DynamoDBTransactHandler[Envelope]) + extends scaladsl.DynamoDBTransactHandler[Envelope] { + + override def process(envelope: Envelope): Future[Iterable[TransactWriteItem]] = + delegate.process(envelope).asScala.map(_.asScala)(ExecutionContext.parasitic) + + override def start(): Future[Done] = + delegate.start().asScala + + override def stop(): Future[Done] = + delegate.stop().asScala +} + +/** + * INTERNAL API: Adapter from javadsl.DynamoDBTransactHandler to scaladsl.DynamoDBTransactHandler + */ +@InternalApi private[projection] class DynamoDBTransactGroupedHandlerAdapter[Envelope]( + delegate: javadsl.DynamoDBTransactHandler[java.util.List[Envelope]]) + extends scaladsl.DynamoDBTransactHandler[Seq[Envelope]] { + + override def process(envelopes: Seq[Envelope]): Future[Iterable[TransactWriteItem]] = + delegate.process(envelopes.asJava).asScala.map(_.asScala)(ExecutionContext.parasitic) + + override def start(): Future[Done] = + delegate.start().asScala + + override def stop(): Future[Done] = + delegate.stop().asScala +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBOffsetStore.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBOffsetStore.scala new file mode 100644 index 000000000..3123d54fb --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBOffsetStore.scala @@ -0,0 +1,733 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.internal + +import java.time.Instant +import java.time.{ Duration => JDuration } +import java.util.concurrent.atomic.AtomicReference + +import scala.annotation.tailrec +import scala.concurrent.ExecutionContext +import scala.concurrent.Future + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.annotation.InternalApi +import akka.persistence.Persistence +import akka.persistence.dynamodb.internal.EnvelopeOrigin +import akka.persistence.query.DeletedDurableState +import akka.persistence.query.DurableStateChange +import akka.persistence.query.TimestampOffset +import akka.persistence.query.TimestampOffsetBySlice +import akka.persistence.query.UpdatedDurableState +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.query.typed.scaladsl.EventTimestampQuery +import akka.projection.BySlicesSourceProvider +import akka.projection.ProjectionId +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.internal.ManagementState +import org.slf4j.LoggerFactory +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + +/** + * INTERNAL API + */ +@InternalApi +private[projection] object DynamoDBOffsetStore { + type SeqNr = Long + type Pid = String + + final case class Record(slice: Int, pid: Pid, seqNr: SeqNr, timestamp: Instant) + final case class RecordWithOffset( + record: Record, + offset: TimestampOffset, + strictSeqNr: Boolean, + fromBacktracking: Boolean, + fromPubSub: Boolean, + fromSnapshot: Boolean) + + object State { + val empty: State = State(Map.empty, Map.empty, Instant.EPOCH, 0) + + def apply(offsetBySlice: Map[Int, TimestampOffset]): State = + if (offsetBySlice.isEmpty) empty + else new State(Map.empty, offsetBySlice, Instant.EPOCH, 0) + + } + + final case class State( + byPid: Map[Pid, Record], + offsetBySlice: Map[Int, TimestampOffset], + oldestTimestamp: Instant, + sizeAfterEvict: Int) { + + def size: Int = byPid.size + + def latestTimestamp: Instant = + if (offsetBySlice.isEmpty) Instant.EPOCH + else offsetBySlice.valuesIterator.map(_.timestamp).max + + def latestOffset: TimestampOffset = + if (offsetBySlice.isEmpty) TimestampOffset.Zero + else offsetBySlice.valuesIterator.maxBy(_.timestamp) + + def add(records: IndexedSeq[Record]): State = { + records.foldLeft(this) { + case (acc, r) => + val newByPid = + acc.byPid.get(r.pid) match { + case Some(existingRecord) => + if (r.seqNr > existingRecord.seqNr) + acc.byPid.updated(r.pid, r) + else + acc.byPid // older or same seqNr + case None => + acc.byPid.updated(r.pid, r) + } + + val newOffsetBySlice = + acc.offsetBySlice.get(r.slice) match { + case Some(existing) => + if (r.timestamp.isAfter(existing.timestamp)) + acc.offsetBySlice.updated(r.slice, TimestampOffset(r.timestamp, Map(r.pid -> r.seqNr))) + else if (r.timestamp == existing.timestamp) + acc.offsetBySlice + .updated(r.slice, TimestampOffset(r.timestamp, existing.seen.updated(r.pid, r.seqNr))) + else + acc.offsetBySlice + case None => + acc.offsetBySlice.updated(r.slice, TimestampOffset(r.timestamp, Map(r.pid -> r.seqNr))) + } + + val newOldestTimestamp = + if (acc.oldestTimestamp == Instant.EPOCH) + r.timestamp // first record + else if (r.timestamp.isBefore(acc.oldestTimestamp)) + r.timestamp + else + acc.oldestTimestamp // this is the normal case + + acc.copy(byPid = newByPid, offsetBySlice = newOffsetBySlice, oldestTimestamp = newOldestTimestamp) + } + } + + def contains(pid: Pid): Boolean = + byPid.contains(pid) + + def isDuplicate(record: Record): Boolean = { + byPid.get(record.pid) match { + case Some(existingRecord) => record.seqNr <= existingRecord.seqNr + case None => false + } + } + + def window: JDuration = + JDuration.between(oldestTimestamp, latestTimestamp) + + private lazy val sortedByTimestamp: Vector[Record] = byPid.valuesIterator.toVector.sortBy(_.timestamp) + + def evict(until: Instant, keepNumberOfEntries: Int): State = { + if (oldestTimestamp.isBefore(until) && size > keepNumberOfEntries) { + val newState = State(offsetBySlice).add( + sortedByTimestamp + .take(size - keepNumberOfEntries) + .filterNot(_.timestamp.isBefore(until)) ++ sortedByTimestamp + .takeRight(keepNumberOfEntries)) + newState.copy(sizeAfterEvict = newState.size) + } else + this + } + + } + + final class RejectedEnvelope(message: String) extends IllegalStateException(message) + + sealed trait Validation + + object Validation { + case object Accepted extends Validation + case object Duplicate extends Validation + case object RejectedSeqNr extends Validation + case object RejectedBacktrackingSeqNr extends Validation + + val FutureAccepted: Future[Validation] = Future.successful(Accepted) + val FutureDuplicate: Future[Validation] = Future.successful(Duplicate) + val FutureRejectedSeqNr: Future[Validation] = Future.successful(RejectedSeqNr) + val FutureRejectedBacktrackingSeqNr: Future[Validation] = Future.successful(RejectedBacktrackingSeqNr) + } + + val FutureDone: Future[Done] = Future.successful(Done) +} + +/** + * INTERNAL API + */ +@InternalApi +private[projection] class DynamoDBOffsetStore( + projectionId: ProjectionId, + sourceProvider: Option[BySlicesSourceProvider], + system: ActorSystem[_], + settings: DynamoDBProjectionSettings, + client: DynamoDbAsyncClient) { + + import DynamoDBOffsetStore._ + + private val persistenceExt = Persistence(system) + + private val (minSlice: Int, maxSlice: Int) = sourceProvider match { + case Some(s) => s.minSlice -> s.maxSlice + case None => 0 -> (persistenceExt.numberOfSlices - 1) + } + + private val logger = LoggerFactory.getLogger(this.getClass) + private val logPrefix = s"${projectionId.name} [$minSlice-$maxSlice]:" + + private val dao = new OffsetStoreDao(system, settings, projectionId, client) + + private val evictWindow = settings.timeWindow.plus(settings.evictInterval) + + private[projection] implicit val executionContext: ExecutionContext = system.executionContext + + // The OffsetStore instance is used by a single projectionId and there shouldn't be many concurrent + // calls to methods that access the `state`, but for example validate (load) may be concurrent + // with save. Therefore, this state can be updated concurrently with CAS retries. + private val state = new AtomicReference(State.empty) + + // Transient state of inflight pid -> seqNr (before they have been stored and included in `state`), which is + // needed for at-least-once or other projections where the offset is saved afterwards. Not needed for exactly-once. + // This can be updated concurrently with CAS retries. + private val inflight = new AtomicReference(Map.empty[Pid, SeqNr]) + + private def timestampOf(persistenceId: String, sequenceNr: Long): Future[Option[Instant]] = { + sourceProvider match { + case Some(timestampQuery: EventTimestampQuery) => + timestampQuery.timestampOf(persistenceId, sequenceNr) + case Some(timestampQuery: akka.persistence.query.typed.javadsl.EventTimestampQuery) => + import scala.jdk.FutureConverters._ + import scala.jdk.OptionConverters._ + timestampQuery.timestampOf(persistenceId, sequenceNr).asScala.map(_.toScala) + case Some(_) => + throw new IllegalArgumentException( + s"Expected BySlicesSourceProvider to implement EventTimestampQuery when TimestampOffset is used.") + case None => + throw new IllegalArgumentException( + s"Expected BySlicesSourceProvider to be defined when TimestampOffset is used.") + } + } + + def getState(): State = + state.get() + + def getInflight(): Map[Pid, SeqNr] = + inflight.get() + + // This is used by projection management and returns latest offset + def getOffset[Offset](): Future[Option[Offset]] = { + // FIXME in r2dbc this will reload via readOffset if no state + Future.successful(Some(getState().latestOffset).map(_.asInstanceOf[Offset])) + } + + def readOffset[Offset](): Future[Option[Offset]] = { + // look for TimestampOffset first since that is used by akka-persistence-dynamodb, + // and then fall back to the other more primitive offset types + sourceProvider match { + case Some(_) => + readTimestampOffset().map { offsetBySlice => + if (offsetBySlice.offsets.isEmpty) None + else Some(offsetBySlice.asInstanceOf[Offset]) + }(ExecutionContext.parasitic) + case None => + // FIXME primitive offsets not supported, maybe we can change the sourceProvider parameter + throw new IllegalStateException("BySlicesSourceProvider is required. Primitive offsets not supported.") + } + } + + private def readTimestampOffset(): Future[TimestampOffsetBySlice] = { + val oldState = state.get() + // retrieve latest timestamp for each slice, and use the earliest + val futTimestamps = + (minSlice to maxSlice).map(slice => dao.loadTimestampOffset(slice).map(optTimestamp => slice -> optTimestamp)) + val offsetBySliceFut = Future.sequence(futTimestamps).map(_.collect { case (slice, Some(ts)) => slice -> ts }.toMap) + offsetBySliceFut.map { offsetBySlice => + val newState = State(offsetBySlice) + logger.debug( + "{} readTimestampOffset state with [{}] persistenceIds, oldest [{}], latest [{}]", + logPrefix, + newState.byPid.size, + newState.oldestTimestamp, + newState.latestTimestamp) + if (!state.compareAndSet(oldState, newState)) + throw new IllegalStateException("Unexpected concurrent modification of state from readOffset.") + clearInflight() + if (offsetBySlice.isEmpty) { + logger.debug("{} readTimestampOffset no stored offset", logPrefix) + TimestampOffsetBySlice.empty + } else { + if (logger.isDebugEnabled) + logger.debug( + "{} readTimestampOffset earliest slice [{}], latest slice [{}]", + logPrefix, + offsetBySlice.minBy(_._1), + offsetBySlice.maxBy(_._1)) + + TimestampOffsetBySlice(offsetBySlice) + } + } + } + + def load(pid: Pid): Future[State] = { + val oldState = state.get() + if (oldState.contains(pid)) + Future.successful(oldState) + else { + val slice = persistenceExt.sliceForPersistenceId(pid) + logger.trace("{} load [{}]", logPrefix, pid) + dao.loadSequenceNumber(slice, pid).flatMap { + case Some(record) => + val newState = oldState.add(Vector(record)) + if (state.compareAndSet(oldState, newState)) + Future.successful(newState) + else + load(pid) // CAS retry, concurrent update + case None => Future.successful(oldState) + } + } + } + + def load(pids: IndexedSeq[Pid]): Future[State] = { + val oldState = state.get() + val pidsToLoad = pids.filterNot(oldState.contains) + if (pidsToLoad.isEmpty) + Future.successful(oldState) + else { + val loadedRecords = pidsToLoad.map { pid => + val slice = persistenceExt.sliceForPersistenceId(pid) + logger.trace("{} load [{}]", logPrefix, pid) + dao.loadSequenceNumber(slice, pid) + } + Future.sequence(loadedRecords).flatMap { records => + val newState = oldState.add(records.flatten) + if (state.compareAndSet(oldState, newState)) + Future.successful(newState) + else + load(pids) // CAS retry, concurrent update + } + } + } + + def saveOffset(offset: OffsetPidSeqNr): Future[Done] = + saveOffsets(Vector(offset)) + + def saveOffsets(offsets: IndexedSeq[OffsetPidSeqNr]): Future[Done] = + storeOffsets(offsets, dao.storeSequenceNumbers, canBeConcurrent = true) + + def transactSaveOffset(writeItems: Iterable[TransactWriteItem], offset: OffsetPidSeqNr): Future[Done] = + transactSaveOffsets(writeItems, Vector(offset)) + + def transactSaveOffsets(writeItems: Iterable[TransactWriteItem], offsets: IndexedSeq[OffsetPidSeqNr]): Future[Done] = + storeOffsets(offsets, dao.transactStoreSequenceNumbers(writeItems), canBeConcurrent = false) + + private def storeOffsets( + offsets: IndexedSeq[OffsetPidSeqNr], + storeSequenceNumbers: IndexedSeq[Record] => Future[Done], + canBeConcurrent: Boolean): Future[Done] = { + if (offsets.isEmpty) + FutureDone + else if (offsets.head.offset.isInstanceOf[TimestampOffset]) { + val records = offsets.map { + case OffsetPidSeqNr(t: TimestampOffset, Some((pid, seqNr))) => + val slice = persistenceExt.sliceForPersistenceId(pid) + Record(slice, pid, seqNr, t.timestamp) + case OffsetPidSeqNr(_: TimestampOffset, None) => + throw new IllegalArgumentException("Required EventEnvelope or DurableStateChange for TimestampOffset.") + case _ => + throw new IllegalArgumentException( + "Mix of TimestampOffset and other offset type in same transaction is not supported") + } + storeTimestampOffsets(records, storeSequenceNumbers, canBeConcurrent) + } else { + throw new IllegalStateException("TimestampOffset is required. Primitive offsets not supported.") + } + } + + private def storeTimestampOffsets( + records: IndexedSeq[Record], + storeSequenceNumbers: IndexedSeq[Record] => Future[Done], + canBeConcurrent: Boolean): Future[Done] = { + load(records.map(_.pid)).flatMap { oldState => + val filteredRecords = { + if (records.size <= 1) + records.filterNot(oldState.isDuplicate) + else { + // use last record for each pid + records + .groupBy(_.pid) + .valuesIterator + .collect { + case recordsByPid if !oldState.isDuplicate(recordsByPid.last) => recordsByPid.last + } + .toVector + } + } + if (filteredRecords.isEmpty) { + FutureDone + } else { + val newState = oldState.add(filteredRecords) + + // accumulate some more than the timeWindow before evicting, and at least 10% increase of size + // for testing keepNumberOfEntries = 0 is used + val evictThresholdReached = + if (settings.keepNumberOfEntries == 0) true else newState.size > (newState.sizeAfterEvict * 1.1).toInt + val evictedNewState = + if (newState.size > settings.keepNumberOfEntries && evictThresholdReached && newState.window + .compareTo(evictWindow) > 0) { + // FIXME maybe this should take the slice into account + val evictUntil = newState.latestTimestamp.minus(settings.timeWindow) + val s = newState.evict(evictUntil, settings.keepNumberOfEntries) + logger.debug( + "{} Evicted [{}] records until [{}], keeping [{}] records. Latest [{}].", + logPrefix, + newState.size - s.size, + evictUntil, + s.size, + newState.latestTimestamp) + s + } else + newState + + // FIXME we probably don't have to store the latest offset per slice all the time, but can + // accumulate some changes and flush on size/time. + val changedOffsetBySlice = evictedNewState.offsetBySlice.filter { + case (slice, offset) => + offset != oldState.offsetBySlice.getOrElse(slice, TimestampOffset.Zero) + } + + storeSequenceNumbers(filteredRecords).flatMap { _ => + val storeOffsetsResult = + if (changedOffsetBySlice.isEmpty) FutureDone else dao.storeTimestampOffsets(changedOffsetBySlice) + storeOffsetsResult.flatMap { _ => + if (state.compareAndSet(oldState, evictedNewState)) { + cleanupInflight(evictedNewState) + FutureDone + } else { // concurrent update + if (canBeConcurrent) storeTimestampOffsets(records, storeSequenceNumbers, canBeConcurrent) // CAS retry + else throw new IllegalStateException("Unexpected concurrent modification of state in save offsets.") + } + } + } + } + } + } + + @tailrec private def cleanupInflight(newState: State): Unit = { + val currentInflight = getInflight() + val newInflight = + currentInflight.filter { + case (inflightPid, inflightSeqNr) => + newState.byPid.get(inflightPid) match { + case Some(r) => r.seqNr < inflightSeqNr + case None => true + } + } + if (newInflight.size >= 10000) { + throw new IllegalStateException( + s"Too many envelopes in-flight [${newInflight.size}]. " + + "Please report this issue at https://github.com/akka/akka-persistence-dynamodb") + } + if (!inflight.compareAndSet(currentInflight, newInflight)) + cleanupInflight(newState) // CAS retry, concurrent update of inflight + } + + @tailrec private def clearInflight(): Unit = { + val currentInflight = getInflight() + if (!inflight.compareAndSet(currentInflight, Map.empty[Pid, SeqNr])) + clearInflight() // CAS retry, concurrent update of inflight + } + + /** + * The stored sequence number for a persistenceId, or 0 if unknown persistenceId. + */ + def storedSeqNr(pid: Pid): Future[SeqNr] = { + getState().byPid.get(pid) match { + case Some(record) => Future.successful(record.seqNr) + case None => + val slice = persistenceExt.sliceForPersistenceId(pid) + dao.loadSequenceNumber(slice, pid).map { + case Some(record) => record.seqNr + case None => 0L + } + } + } + + def validateAll[Envelope](envelopes: Seq[Envelope]): Future[Seq[(Envelope, Validation)]] = { + import Validation._ + + envelopes + .foldLeft(Future.successful((getInflight(), Vector.empty[(Envelope, Validation)]))) { (acc, envelope) => + acc.flatMap { + case (inflight, filteredEnvelopes) => + createRecordWithOffset(envelope) match { + case Some(recordWithOffset) => + validate(recordWithOffset, inflight).map { + case Accepted => + ( + inflight.updated(recordWithOffset.record.pid, recordWithOffset.record.seqNr), + filteredEnvelopes :+ (envelope -> Accepted)) + case rejected => + (inflight, filteredEnvelopes :+ (envelope -> rejected)) + } + case None => + Future.successful((inflight, filteredEnvelopes :+ (envelope -> Accepted))) + } + } + } + .map { + case (_, filteredEnvelopes) => + filteredEnvelopes + } + } + + /** + * Validate if the sequence number of the envelope is the next expected, or if the envelope is a duplicate that has + * already been processed, or there is a gap in sequence numbers that should be rejected. + */ + def validate[Envelope](envelope: Envelope): Future[Validation] = { + createRecordWithOffset(envelope) match { + case Some(recordWithOffset) => validate(recordWithOffset, getInflight()) + case None => Validation.FutureAccepted + } + } + + private def validate(recordWithOffset: RecordWithOffset, currentInflight: Map[Pid, SeqNr]): Future[Validation] = { + import Validation._ + val pid = recordWithOffset.record.pid + val seqNr = recordWithOffset.record.seqNr + + load(pid).flatMap { currentState => + val duplicate = currentState.isDuplicate(recordWithOffset.record) + + if (duplicate) { + logger.trace("{} Filtering out duplicate sequence number [{}] for pid [{}]", logPrefix, seqNr, pid) + FutureDuplicate + } else if (recordWithOffset.strictSeqNr) { + // strictSeqNr == true is for event sourced + val prevSeqNr = currentInflight.getOrElse(pid, currentState.byPid.get(pid).map(_.seqNr).getOrElse(0L)) + + def logUnexpected(): Unit = { + if (recordWithOffset.fromPubSub) + logger.debug( + "{} Rejecting pub-sub envelope, unexpected sequence number [{}] for pid [{}], previous sequence number [{}]. Offset: {}", + logPrefix, + seqNr, + pid, + prevSeqNr, + recordWithOffset.offset) + else if (!recordWithOffset.fromBacktracking) + logger.debug( + "{} Rejecting unexpected sequence number [{}] for pid [{}], previous sequence number [{}]. Offset: {}", + logPrefix, + seqNr, + pid, + prevSeqNr, + recordWithOffset.offset) + else + logger.warn( + "{} Rejecting unexpected sequence number [{}] for pid [{}], previous sequence number [{}]. Offset: {}", + logPrefix, + seqNr, + pid, + prevSeqNr, + recordWithOffset.offset) + } + + def logUnknown(): Unit = { + if (recordWithOffset.fromPubSub) { + logger.debug( + "{} Rejecting pub-sub envelope, unknown sequence number [{}] for pid [{}] (might be accepted later): {}", + logPrefix, + seqNr, + pid, + recordWithOffset.offset) + } else if (!recordWithOffset.fromBacktracking) { + // This may happen rather frequently when using `publish-events`, after reconnecting and such. + logger.debug( + "{} Rejecting unknown sequence number [{}] for pid [{}] (might be accepted later): {}", + logPrefix, + seqNr, + pid, + recordWithOffset.offset) + } else { + logger.warn( + "{} Rejecting unknown sequence number [{}] for pid [{}]. Offset: {}", + logPrefix, + seqNr, + pid, + recordWithOffset.offset) + } + } + + if (prevSeqNr > 0) { + // expecting seqNr to be +1 of previously known + val ok = seqNr == prevSeqNr + 1 + if (ok) { + FutureAccepted + } else if (seqNr <= currentInflight.getOrElse(pid, 0L)) { + // currentInFlight contains those that have been processed or about to be processed in Flow, + // but offset not saved yet => ok to handle as duplicate + FutureDuplicate + } else if (recordWithOffset.fromSnapshot) { + // snapshots will mean we are starting from some arbitrary offset after last seen offset + FutureAccepted + } else if (!recordWithOffset.fromBacktracking) { + logUnexpected() + FutureRejectedSeqNr + } else { + logUnexpected() + // This will result in projection restart (with normal configuration) + FutureRejectedBacktrackingSeqNr + } + } else if (seqNr == 1) { + // always accept first event if no other event for that pid has been seen + FutureAccepted + } else if (recordWithOffset.fromSnapshot) { + // always accept starting from snapshots when there was no previous event seen + FutureAccepted + } else { + // Haven't see seen this pid within the time window. Since events can be missed + // when read at the tail we will only accept it if the event with previous seqNr has timestamp + // before the time window of the offset store. + // Backtracking will emit missed event again. + timestampOf(pid, seqNr - 1).map { + case Some(previousTimestamp) => + val before = currentState.latestTimestamp.minus(settings.timeWindow) + if (previousTimestamp.isBefore(before)) { + logger.debug( + "{} Accepting envelope with pid [{}], seqNr [{}], where previous event timestamp [{}] " + + "is before time window [{}].", + logPrefix, + pid, + seqNr, + previousTimestamp, + before) + Accepted + } else if (!recordWithOffset.fromBacktracking) { + logUnknown() + RejectedSeqNr + } else { + logUnknown() + // This will result in projection restart (with normal configuration) + RejectedBacktrackingSeqNr + } + case None => + // previous not found, could have been deleted + Accepted + } + } + } else { + // strictSeqNr == false is for durable state where each revision might not be visible + val prevSeqNr = currentInflight.getOrElse(pid, currentState.byPid.get(pid).map(_.seqNr).getOrElse(0L)) + val ok = seqNr > prevSeqNr + + if (ok) { + FutureAccepted + } else { + logger.trace( + "{} Filtering out earlier revision [{}] for pid [{}], previous revision [{}]", + logPrefix, + seqNr, + pid, + prevSeqNr) + FutureDuplicate + } + } + } + } + + @tailrec final def addInflight[Envelope](envelope: Envelope): Unit = { + createRecordWithOffset(envelope) match { + case Some(recordWithOffset) => + val currentInflight = getInflight() + val newInflight = currentInflight.updated(recordWithOffset.record.pid, recordWithOffset.record.seqNr) + if (!inflight.compareAndSet(currentInflight, newInflight)) + addInflight(envelope) // CAS retry, concurrent update of inflight + case None => + } + } + + @tailrec final def addInflights[Envelope](envelopes: Seq[Envelope]): Unit = { + val currentInflight = getInflight() + val entries = envelopes.iterator.map(createRecordWithOffset).collect { + case Some(r) => + r.record.pid -> r.record.seqNr + } + val newInflight = currentInflight ++ entries + if (!inflight.compareAndSet(currentInflight, newInflight)) + addInflights(envelopes) // CAS retry, concurrent update of inflight + } + + def isInflight[Envelope](envelope: Envelope): Boolean = { + createRecordWithOffset(envelope) match { + case Some(recordWithOffset) => + val pid = recordWithOffset.record.pid + val seqNr = recordWithOffset.record.seqNr + getInflight().get(pid) match { + case Some(`seqNr`) => true + case _ => false + } + case None => true + } + } + + private def createRecordWithOffset[Envelope](envelope: Envelope): Option[RecordWithOffset] = { + envelope match { + case eventEnvelope: EventEnvelope[_] if eventEnvelope.offset.isInstanceOf[TimestampOffset] => + val timestampOffset = eventEnvelope.offset.asInstanceOf[TimestampOffset] + val slice = persistenceExt.sliceForPersistenceId(eventEnvelope.persistenceId) + Some( + RecordWithOffset( + Record(slice, eventEnvelope.persistenceId, eventEnvelope.sequenceNr, timestampOffset.timestamp), + timestampOffset, + strictSeqNr = true, + fromBacktracking = EnvelopeOrigin.fromBacktracking(eventEnvelope), + fromPubSub = EnvelopeOrigin.fromPubSub(eventEnvelope), + fromSnapshot = EnvelopeOrigin.fromSnapshot(eventEnvelope))) + case change: UpdatedDurableState[_] if change.offset.isInstanceOf[TimestampOffset] => + val timestampOffset = change.offset.asInstanceOf[TimestampOffset] + val slice = persistenceExt.sliceForPersistenceId(change.persistenceId) + Some( + RecordWithOffset( + Record(slice, change.persistenceId, change.revision, timestampOffset.timestamp), + timestampOffset, + strictSeqNr = false, + fromBacktracking = EnvelopeOrigin.fromBacktracking(change), + fromPubSub = false, + fromSnapshot = false)) + case change: DeletedDurableState[_] if change.offset.isInstanceOf[TimestampOffset] => + val timestampOffset = change.offset.asInstanceOf[TimestampOffset] + val slice = persistenceExt.sliceForPersistenceId(change.persistenceId) + Some( + RecordWithOffset( + Record(slice, change.persistenceId, change.revision, timestampOffset.timestamp), + timestampOffset, + strictSeqNr = false, + fromBacktracking = false, + fromPubSub = false, + fromSnapshot = false)) + case change: DurableStateChange[_] if change.offset.isInstanceOf[TimestampOffset] => + // in case additional types are added + throw new IllegalArgumentException( + s"DurableStateChange [${change.getClass.getName}] not implemented yet. Please report bug at https://github.com/akka/akka-projection/issues") + case _ => None + } + } + + def readManagementState(): Future[Option[ManagementState]] = + dao.readManagementState(minSlice) + + def savePaused(paused: Boolean): Future[Done] = + dao.updateManagementState(minSlice, maxSlice, paused) + +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBProjectionImpl.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBProjectionImpl.scala new file mode 100644 index 000000000..a60d574df --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/DynamoDBProjectionImpl.scala @@ -0,0 +1,704 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.internal + +import java.util.concurrent.atomic.AtomicLong + +import scala.annotation.nowarn +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.concurrent.duration.FiniteDuration +import scala.util.control.NonFatal + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.annotation.InternalApi +import akka.event.Logging +import akka.event.LoggingAdapter +import akka.persistence.query.DeletedDurableState +import akka.persistence.query.DurableStateChange +import akka.persistence.query.UpdatedDurableState +import akka.persistence.query.typed.EventEnvelope +import akka.persistence.query.typed.scaladsl.LoadEventQuery +import akka.persistence.state.scaladsl.DurableStateStore +import akka.persistence.state.scaladsl.GetObjectResult +import akka.projection.BySlicesSourceProvider +import akka.projection.HandlerRecoveryStrategy +import akka.projection.HandlerRecoveryStrategy.Internal.RetryAndSkip +import akka.projection.HandlerRecoveryStrategy.Internal.Skip +import akka.projection.ProjectionContext +import akka.projection.ProjectionId +import akka.projection.RunningProjection +import akka.projection.RunningProjection.AbortProjectionException +import akka.projection.RunningProjectionManagement +import akka.projection.StatusObserver +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.RejectedEnvelope +import akka.projection.dynamodb.scaladsl.DynamoDBTransactHandler +import akka.projection.internal.ActorHandlerInit +import akka.projection.internal.AtLeastOnce +import akka.projection.internal.AtMostOnce +import akka.projection.internal.CanTriggerReplay +import akka.projection.internal.ExactlyOnce +import akka.projection.internal.GroupedHandlerStrategy +import akka.projection.internal.HandlerStrategy +import akka.projection.internal.InternalProjection +import akka.projection.internal.InternalProjectionState +import akka.projection.internal.ManagementState +import akka.projection.internal.OffsetStoredByHandler +import akka.projection.internal.OffsetStrategy +import akka.projection.internal.ProjectionContextImpl +import akka.projection.internal.ProjectionSettings +import akka.projection.internal.SettingsImpl +import akka.projection.javadsl +import akka.projection.scaladsl +import akka.projection.scaladsl.Handler +import akka.projection.scaladsl.SourceProvider +import akka.stream.RestartSettings +import akka.stream.scaladsl.FlowWithContext +import akka.stream.scaladsl.Source +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient + +/** + * INTERNAL API + */ +@InternalApi +private[projection] object DynamoDBProjectionImpl { + import akka.persistence.dynamodb.internal.EnvelopeOrigin.fromBacktracking + import akka.persistence.dynamodb.internal.EnvelopeOrigin.isFilteredEvent + + val log: Logger = LoggerFactory.getLogger(classOf[DynamoDBProjectionImpl[_, _]]) + + private val FutureDone: Future[Done] = Future.successful(Done) + private val FutureFalse: Future[Boolean] = Future.successful(false) + + private[projection] def createOffsetStore( + projectionId: ProjectionId, + sourceProvider: Option[BySlicesSourceProvider], + settings: DynamoDBProjectionSettings, + client: DynamoDbAsyncClient)(implicit system: ActorSystem[_]) = { + new DynamoDBOffsetStore(projectionId, sourceProvider, system, settings, client) + } + + private val loadEnvelopeCounter = new AtomicLong + + def loadEnvelope[Envelope](env: Envelope, sourceProvider: SourceProvider[_, Envelope])( + implicit + ec: ExecutionContext): Future[Envelope] = { + env match { + case eventEnvelope: EventEnvelope[_] + if fromBacktracking(eventEnvelope) && eventEnvelope.eventOption.isEmpty && !eventEnvelope.filtered => + val pid = eventEnvelope.persistenceId + val seqNr = eventEnvelope.sequenceNr + (sourceProvider match { + case loadEventQuery: LoadEventQuery => + loadEventQuery.loadEnvelope[Any](pid, seqNr) + case loadEventQuery: akka.persistence.query.typed.javadsl.LoadEventQuery => + import scala.jdk.FutureConverters._ + loadEventQuery.loadEnvelope[Any](pid, seqNr).asScala + case _ => + throw new IllegalArgumentException( + s"Expected sourceProvider [${sourceProvider.getClass.getName}] " + + "to implement LoadEventQuery when used with eventsBySlices.") + }).map { loadedEnv => + val count = loadEnvelopeCounter.incrementAndGet() + if (count % 1000 == 0) + log.info("Loaded event lazily, persistenceId [{}], seqNr [{}]. Load count [{}]", pid, seqNr, count) + else + log.debug("Loaded event lazily, persistenceId [{}], seqNr [{}]. Load count [{}]", pid, seqNr, count) + loadedEnv.asInstanceOf[Envelope] + } + + case upd: UpdatedDurableState[_] if upd.value == null => + val pid = upd.persistenceId + (sourceProvider match { + case store: DurableStateStore[_] => + store.getObject(pid) + case store: akka.persistence.state.javadsl.DurableStateStore[_] => + import scala.jdk.FutureConverters._ + store.getObject(pid).asScala.map(_.toScala) + case unknown => + throw new IllegalArgumentException(s"Unsupported source provider type '${unknown.getClass}'") + }).map { + case GetObjectResult(Some(loadedValue), loadedRevision) => + val count = loadEnvelopeCounter.incrementAndGet() + if (count % 1000 == 0) + log.info( + "Loaded durable state lazily, persistenceId [{}], revision [{}]. Load count [{}]", + pid, + loadedRevision, + count) + else + log.debug( + "Loaded durable state lazily, persistenceId [{}], revision [{}]. Load count [{}]", + pid, + loadedRevision, + count) + new UpdatedDurableState(pid, loadedRevision, loadedValue, upd.offset, upd.timestamp) + .asInstanceOf[Envelope] + case GetObjectResult(None, loadedRevision) => + new DeletedDurableState(pid, loadedRevision, upd.offset, upd.timestamp) + .asInstanceOf[Envelope] + } + + case _ => + Future.successful(env) + } + } + + private def extractOffsetPidSeqNr[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + envelope: Envelope): OffsetPidSeqNr = + extractOffsetPidSeqNr(sourceProvider.extractOffset(envelope), envelope) + + private def extractOffsetPidSeqNr[Offset, Envelope](offset: Offset, envelope: Envelope): OffsetPidSeqNr = { + // we could define a new trait for the SourceProvider to implement this in case other (custom) envelope types are needed + envelope match { + case env: EventEnvelope[_] => OffsetPidSeqNr(offset, env.persistenceId, env.sequenceNr) + case chg: UpdatedDurableState[_] => OffsetPidSeqNr(offset, chg.persistenceId, chg.revision) + case del: DeletedDurableState[_] => OffsetPidSeqNr(offset, del.persistenceId, del.revision) + case other => // avoid unreachable error on sealed DurableStateChange + other match { + case change: DurableStateChange[_] => + // in case additional types are added + throw new IllegalArgumentException( + s"DurableStateChange [${change.getClass.getName}] not implemented yet. Please report bug at https://github.com/akka/akka-projection/issues") + case _ => OffsetPidSeqNr(offset) + } + } + } + + private[projection] def adaptedHandlerForAtLeastOnce[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + handlerFactory: () => Handler[Envelope], + offsetStore: DynamoDBOffsetStore)( + implicit + ec: ExecutionContext, + system: ActorSystem[_]): () => Handler[Envelope] = { () => + new AdaptedHandler(handlerFactory()) { + override def process(envelope: Envelope): Future[Done] = { + import DynamoDBOffsetStore.Validation._ + offsetStore + .validate(envelope) + .flatMap { + case Accepted => + if (isFilteredEvent(envelope)) { + offsetStore.addInflight(envelope) + FutureDone + } else { + loadEnvelope(envelope, sourceProvider).flatMap { loadedEnvelope => + delegate + .process(loadedEnvelope) + .map { _ => + offsetStore.addInflight(loadedEnvelope) + Done + } + } + } + case Duplicate => + FutureDone + case RejectedSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, envelope).map(_ => Done)(ExecutionContext.parasitic) + case RejectedBacktrackingSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, envelope).map { + case true => Done + case false => throwRejectedEnvelope(sourceProvider, envelope) + } + } + } + } + } + + private[projection] def adaptedHandlerForExactlyOnce[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + handlerFactory: () => DynamoDBTransactHandler[Envelope], + offsetStore: DynamoDBOffsetStore)( + implicit + ec: ExecutionContext, + system: ActorSystem[_]): () => Handler[Envelope] = { () => + + new AdaptedDynamoDBTransactHandler(handlerFactory()) { + override def process(envelope: Envelope): Future[Done] = { + import DynamoDBOffsetStore.Validation._ + offsetStore + .validate(envelope) + .flatMap { + case Accepted => + if (isFilteredEvent(envelope)) { + val offset = extractOffsetPidSeqNr(sourceProvider, envelope) + offsetStore.saveOffset(offset) + } else { + loadEnvelope(envelope, sourceProvider).flatMap { loadedEnvelope => + val offset = extractOffsetPidSeqNr(sourceProvider, loadedEnvelope) + delegate.process(loadedEnvelope).flatMap { writeItems => + offsetStore.transactSaveOffset(writeItems, offset) + } + } + } + case Duplicate => + FutureDone + case RejectedSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, envelope).map(_ => Done)(ExecutionContext.parasitic) + case RejectedBacktrackingSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, envelope).map { + case true => Done + case false => throwRejectedEnvelope(sourceProvider, envelope) + } + } + } + } + } + + private[projection] def adaptedHandlerForExactlyOnceGrouped[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + handlerFactory: () => DynamoDBTransactHandler[Seq[Envelope]], + offsetStore: DynamoDBOffsetStore)( + implicit + ec: ExecutionContext, + system: ActorSystem[_]): () => Handler[Seq[Envelope]] = { () => + + new AdaptedDynamoDBTransactHandler(handlerFactory()) { + override def process(envelopes: Seq[Envelope]): Future[Done] = { + import DynamoDBOffsetStore.Validation._ + offsetStore.validateAll(envelopes).flatMap { isAcceptedEnvelopes => + val replayDone = + Future.sequence(isAcceptedEnvelopes.map { + case (env, RejectedSeqNr) => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map(_ => Done)(ExecutionContext.parasitic) + case (env, RejectedBacktrackingSeqNr) => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map { + case true => Done + case false => throwRejectedEnvelope(sourceProvider, env) + } + case _ => + FutureDone + }) + + replayDone.flatMap { _ => + val acceptedEnvelopes = isAcceptedEnvelopes.collect { + case (env, Accepted) => + env + } + + if (acceptedEnvelopes.isEmpty) { + FutureDone + } else { + Future.sequence(acceptedEnvelopes.map(env => loadEnvelope(env, sourceProvider))).flatMap { + loadedEnvelopes => + val offsets = loadedEnvelopes.iterator.map(extractOffsetPidSeqNr(sourceProvider, _)).toVector + val filteredEnvelopes = loadedEnvelopes.filterNot(isFilteredEvent) + if (filteredEnvelopes.isEmpty) { + offsetStore.saveOffsets(offsets) + } else { + delegate.process(filteredEnvelopes).flatMap { writeItems => + offsetStore.transactSaveOffsets(writeItems, offsets) + } + } + } + } + } + } + } + } + } + + private[projection] def adaptedHandlerForAtLeastOnceGrouped[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + handlerFactory: () => Handler[Seq[Envelope]], + offsetStore: DynamoDBOffsetStore)( + implicit + ec: ExecutionContext, + system: ActorSystem[_]): () => Handler[Seq[Envelope]] = { () => + + new AdaptedHandler(handlerFactory()) { + override def process(envelopes: Seq[Envelope]): Future[Done] = { + import DynamoDBOffsetStore.Validation._ + offsetStore.validateAll(envelopes).flatMap { isAcceptedEnvelopes => + val replayDone = + Future.sequence(isAcceptedEnvelopes.map { + case (env, RejectedSeqNr) => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map(_ => Done)(ExecutionContext.parasitic) + case (env, RejectedBacktrackingSeqNr) => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map { + case true => Done + case false => throwRejectedEnvelope(sourceProvider, env) + } + case _ => + FutureDone + }) + + replayDone.flatMap { _ => + val acceptedEnvelopes = isAcceptedEnvelopes.collect { + case (env, Accepted) => + env + } + + if (acceptedEnvelopes.isEmpty) { + FutureDone + } else { + Future.sequence(acceptedEnvelopes.map(env => loadEnvelope(env, sourceProvider))).flatMap { + loadedEnvelopes => + val offsets = loadedEnvelopes.iterator.map(extractOffsetPidSeqNr(sourceProvider, _)).toVector + val filteredEnvelopes = loadedEnvelopes.filterNot(isFilteredEvent) + if (filteredEnvelopes.isEmpty) { + offsetStore.saveOffsets(offsets) + } else { + delegate.process(filteredEnvelopes).flatMap { _ => + offsetStore.saveOffsets(offsets) + } + } + } + } + } + } + } + } + } + + private[projection] def adaptedHandlerForFlow[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + handler: FlowWithContext[Envelope, ProjectionContext, Done, ProjectionContext, _], + offsetStore: DynamoDBOffsetStore, + settings: DynamoDBProjectionSettings)( + implicit + system: ActorSystem[_]): FlowWithContext[Envelope, ProjectionContext, Done, ProjectionContext, _] = { + import DynamoDBOffsetStore.Validation._ + implicit val ec: ExecutionContext = system.executionContext + FlowWithContext[Envelope, ProjectionContext] + .mapAsync(1) { env => + offsetStore + .validate(env) + .flatMap { + case Accepted => + if (isFilteredEvent(env) && settings.warnAboutFilteredEventsInFlow) { + log.info("atLeastOnceFlow doesn't support skipping envelopes. Envelope [{}] still emitted.", env) + } + loadEnvelope(env, sourceProvider).map { loadedEnvelope => + offsetStore.addInflight(loadedEnvelope) + Some(loadedEnvelope) + } + case Duplicate => + Future.successful(None) + case RejectedSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map(_ => None)(ExecutionContext.parasitic) + case RejectedBacktrackingSeqNr => + triggerReplayIfPossible(sourceProvider, offsetStore, env).map { + case true => None + case false => throwRejectedEnvelope(sourceProvider, env) + } + } + } + .collect { + case Some(env) => + env + } + .via(handler) + } + + private def triggerReplayIfPossible[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + offsetStore: DynamoDBOffsetStore, + envelope: Envelope)(implicit ec: ExecutionContext): Future[Boolean] = { + envelope match { + case env: EventEnvelope[Any @unchecked] if env.sequenceNr > 1 => + sourceProvider match { + case provider: CanTriggerReplay => + offsetStore.storedSeqNr(env.persistenceId).map { storedSeqNr => + val fromSeqNr = storedSeqNr + 1 + provider.triggerReplay(env.persistenceId, fromSeqNr, env.sequenceNr) + true + } + case _ => + FutureFalse // no replay support for other source providers + } + case _ => + FutureFalse // no replay support for non typed envelopes + } + } + + private def throwRejectedEnvelope[Offset, Envelope]( + sourceProvider: SourceProvider[Offset, Envelope], + envelope: Envelope): Nothing = { + extractOffsetPidSeqNr(sourceProvider, envelope) match { + case OffsetPidSeqNr(_, Some((pid, seqNr))) => + throw new RejectedEnvelope( + s"Rejected envelope from backtracking, persistenceId [$pid], seqNr [$seqNr] due to unexpected sequence number.") + case OffsetPidSeqNr(_, None) => + throw new RejectedEnvelope(s"Rejected envelope from backtracking.") + } + } + + @nowarn("msg=never used") + abstract class AdaptedHandler[E](val delegate: Handler[E])(implicit ec: ExecutionContext, system: ActorSystem[_]) + extends Handler[E] { + + override def start(): Future[Done] = + delegate.start() + + override def stop(): Future[Done] = + delegate.stop() + } + + @nowarn("msg=never used") + abstract class AdaptedDynamoDBTransactHandler[E](val delegate: DynamoDBTransactHandler[E])( + implicit + ec: ExecutionContext, + system: ActorSystem[_]) + extends Handler[E] { + + override def start(): Future[Done] = + delegate.start() + + override def stop(): Future[Done] = + delegate.stop() + } + +} + +/** + * INTERNAL API + */ +@InternalApi +private[projection] class DynamoDBProjectionImpl[Offset, Envelope]( + val projectionId: ProjectionId, + dynamodbSettings: DynamoDBProjectionSettings, + settingsOpt: Option[ProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + restartBackoffOpt: Option[RestartSettings], + val offsetStrategy: OffsetStrategy, + handlerStrategy: HandlerStrategy, + override val statusObserver: StatusObserver[Envelope], + offsetStore: DynamoDBOffsetStore) + extends scaladsl.AtLeastOnceProjection[Offset, Envelope] + with javadsl.AtLeastOnceProjection[Offset, Envelope] + with scaladsl.ExactlyOnceProjection[Offset, Envelope] + with javadsl.ExactlyOnceProjection[Offset, Envelope] + with scaladsl.GroupedProjection[Offset, Envelope] + with javadsl.GroupedProjection[Offset, Envelope] + with scaladsl.AtLeastOnceFlowProjection[Offset, Envelope] + with javadsl.AtLeastOnceFlowProjection[Offset, Envelope] + with SettingsImpl[DynamoDBProjectionImpl[Offset, Envelope]] + with InternalProjection { + import DynamoDBProjectionImpl.extractOffsetPidSeqNr + + private def copy( + settingsOpt: Option[ProjectionSettings] = this.settingsOpt, + restartBackoffOpt: Option[RestartSettings] = this.restartBackoffOpt, + offsetStrategy: OffsetStrategy = this.offsetStrategy, + handlerStrategy: HandlerStrategy = this.handlerStrategy, + statusObserver: StatusObserver[Envelope] = this.statusObserver): DynamoDBProjectionImpl[Offset, Envelope] = + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt, + sourceProvider, + restartBackoffOpt, + offsetStrategy, + handlerStrategy, + statusObserver, + offsetStore) + + type ReadOffset = () => Future[Option[Offset]] + + /* + * Build the final ProjectionSettings to use, if currently set to None fallback to values in config file + */ + private def settingsOrDefaults(implicit system: ActorSystem[_]): ProjectionSettings = { + val settings = settingsOpt.getOrElse(ProjectionSettings(system)) + restartBackoffOpt match { + case None => settings + case Some(r) => settings.copy(restartBackoff = r) + } + } + + override def withRestartBackoffSettings(restartBackoff: RestartSettings): DynamoDBProjectionImpl[Offset, Envelope] = + copy(restartBackoffOpt = Some(restartBackoff)) + + override def withSaveOffset( + afterEnvelopes: Int, + afterDuration: FiniteDuration): DynamoDBProjectionImpl[Offset, Envelope] = + copy(offsetStrategy = offsetStrategy + .asInstanceOf[AtLeastOnce] + .copy(afterEnvelopes = Some(afterEnvelopes), orAfterDuration = Some(afterDuration))) + + override def withGroup( + groupAfterEnvelopes: Int, + groupAfterDuration: FiniteDuration): DynamoDBProjectionImpl[Offset, Envelope] = + copy(handlerStrategy = handlerStrategy + .asInstanceOf[GroupedHandlerStrategy[Envelope]] + .copy(afterEnvelopes = Some(groupAfterEnvelopes), orAfterDuration = Some(groupAfterDuration))) + + override def withRecoveryStrategy( + recoveryStrategy: HandlerRecoveryStrategy): DynamoDBProjectionImpl[Offset, Envelope] = { + val newStrategy = offsetStrategy match { + case s: ExactlyOnce => s.copy(recoveryStrategy = Some(recoveryStrategy)) + case s: AtLeastOnce => s.copy(recoveryStrategy = Some(recoveryStrategy)) + case s: OffsetStoredByHandler => s.copy(recoveryStrategy = Some(recoveryStrategy)) + //NOTE: AtMostOnce has its own withRecoveryStrategy variant + // this method is not available for AtMostOnceProjection + case s: AtMostOnce => s + } + copy(offsetStrategy = newStrategy) + } + + override def withStatusObserver(observer: StatusObserver[Envelope]): DynamoDBProjectionImpl[Offset, Envelope] = + copy(statusObserver = observer) + + private[projection] def actorHandlerInit[T]: Option[ActorHandlerInit[T]] = + handlerStrategy.actorHandlerInit + + /** + * INTERNAL API Return a RunningProjection + */ + override private[projection] def run()(implicit system: ActorSystem[_]): RunningProjection = + new DynamoDBInternalProjectionState(settingsOrDefaults).newRunningInstance() + + /** + * INTERNAL API + * + * This method returns the projection Source mapped with user 'handler' function, but before any sink attached. This + * is mainly intended to be used by the TestKit allowing it to attach a TestSink to it. + */ + override private[projection] def mappedSource()(implicit system: ActorSystem[_]): Source[Done, Future[Done]] = + new DynamoDBInternalProjectionState(settingsOrDefaults).mappedSource() + + private class DynamoDBInternalProjectionState(settings: ProjectionSettings)(implicit val system: ActorSystem[_]) + extends InternalProjectionState[Offset, Envelope]( + projectionId, + sourceProvider, + offsetStrategy, + handlerStrategy, + statusObserver, + settings) { + + implicit val executionContext: ExecutionContext = system.executionContext + override val logger: LoggingAdapter = Logging(system.classicSystem, classOf[DynamoDBProjectionImpl[_, _]]) + + private val isExactlyOnceWithSkip: Boolean = + offsetStrategy match { + case ExactlyOnce(Some(Skip)) | ExactlyOnce(Some(_: RetryAndSkip)) => true + case _ => false + } + + override def readPaused(): Future[Boolean] = + offsetStore.readManagementState().map(_.exists(_.paused)) + + override def readOffsets(): Future[Option[Offset]] = + offsetStore.readOffset() + + // Called from InternalProjectionState.saveOffsetAndReport + override def saveOffset(projectionId: ProjectionId, offset: Offset): Future[Done] = { + // need the envelope to be able to call offsetStore.saveOffset + throw new IllegalStateException( + "Unexpected call to saveOffset. It should have called saveOffsetAndReport. Please report bug at https://github.com/akka/akka-projection/issues") + } + + override protected def saveOffsetAndReport( + projectionId: ProjectionId, + projectionContext: ProjectionContextImpl[Offset, Envelope], + batchSize: Int): Future[Done] = { + import DynamoDBProjectionImpl.FutureDone + val envelope = projectionContext.envelope + + if (offsetStore.isInflight(envelope) || isExactlyOnceWithSkip) { + val offset = extractOffsetPidSeqNr(projectionContext.offset, envelope) + offsetStore + .saveOffset(offset) + .map { done => + try { + statusObserver.offsetProgress(projectionId, envelope) + } catch { + case NonFatal(_) => // ignore + } + getTelemetry().onOffsetStored(batchSize) + done + } + + } else { + FutureDone + } + } + + override protected def saveOffsetsAndReport( + projectionId: ProjectionId, + batch: Seq[ProjectionContextImpl[Offset, Envelope]]): Future[Done] = { + import DynamoDBProjectionImpl.FutureDone + + val acceptedContexts = + if (isExactlyOnceWithSkip) + batch.toVector + else { + batch.iterator.filter { ctx => + val env = ctx.envelope + offsetStore.isInflight(env) + }.toVector + } + + if (acceptedContexts.isEmpty) { + FutureDone + } else { + val offsets = acceptedContexts.map(ctx => extractOffsetPidSeqNr(ctx.offset, ctx.envelope)) + offsetStore + .saveOffsets(offsets) + .map { done => + val batchSize = acceptedContexts.map { _.groupSize }.sum + val last = acceptedContexts.last + try { + statusObserver.offsetProgress(projectionId, last.envelope) + } catch { + case NonFatal(_) => // ignore + } + getTelemetry().onOffsetStored(batchSize) + done + } + } + } + + private[projection] def newRunningInstance(): RunningProjection = + new DynamoDBRunningProjection(RunningProjection.withBackoff(() => this.mappedSource(), settings), this) + } + + private class DynamoDBRunningProjection(source: Source[Done, _], projectionState: DynamoDBInternalProjectionState)( + implicit system: ActorSystem[_]) + extends RunningProjection + with RunningProjectionManagement[Offset] { + + private val streamDone = source.run() + + override def stop(): Future[Done] = { + projectionState.killSwitch.shutdown() + // if the handler is retrying it will be aborted by this, + // otherwise the stream would not be completed by the killSwitch until after all retries + projectionState.abort.failure(AbortProjectionException) + streamDone + } + + // RunningProjectionManagement + override def getOffset(): Future[Option[Offset]] = { + offsetStore.getOffset() + } + + // RunningProjectionManagement + override def setOffset(offset: Option[Offset]): Future[Done] = { + // FIXME + // offset match { + // case Some(o) => offsetStore.managementSetOffset(o) + // case None => offsetStore.managementClearOffset() + //} + ??? + } + + // RunningProjectionManagement + override def getManagementState(): Future[Option[ManagementState]] = + offsetStore.readManagementState() + + // RunningProjectionManagement + override def setPaused(paused: Boolean): Future[Done] = + offsetStore.savePaused(paused) + } + +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetPidSeqNr.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetPidSeqNr.scala new file mode 100644 index 000000000..018665338 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetPidSeqNr.scala @@ -0,0 +1,23 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.internal + +import akka.annotation.InternalApi + +/** + * INTERNAL API + */ +@InternalApi private[akka] object OffsetPidSeqNr { + def apply(offset: Any, pid: String, seqNr: Long): OffsetPidSeqNr = + new OffsetPidSeqNr(offset, Some(pid -> seqNr)) + + def apply(offset: Any): OffsetPidSeqNr = + new OffsetPidSeqNr(offset, None) +} + +/** + * INTERNAL API + */ +@InternalApi private[akka] final case class OffsetPidSeqNr(offset: Any, pidSeqNr: Option[(String, Long)]) diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetStoreDao.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetStoreDao.scala new file mode 100644 index 000000000..40d221da1 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/internal/OffsetStoreDao.scala @@ -0,0 +1,440 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.internal + +import java.time.Instant +import java.util.Collections +import java.util.concurrent.CompletionException +import java.util.{ HashMap => JHashMap } + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.jdk.CollectionConverters._ +import scala.jdk.FutureConverters._ + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.annotation.InternalApi +import akka.persistence.dynamodb.internal.InstantFactory +import akka.persistence.query.TimestampOffset +import akka.projection.ProjectionId +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.dynamodb.internal.DynamoDBOffsetStore.Record +import akka.projection.internal.ManagementState +import org.slf4j.Logger +import org.slf4j.LoggerFactory +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient +import software.amazon.awssdk.services.dynamodb.model.AttributeValue +import software.amazon.awssdk.services.dynamodb.model.BatchWriteItemRequest +import software.amazon.awssdk.services.dynamodb.model.Put +import software.amazon.awssdk.services.dynamodb.model.PutRequest +import software.amazon.awssdk.services.dynamodb.model.QueryRequest +import software.amazon.awssdk.services.dynamodb.model.ReturnConsumedCapacity +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItemsRequest +import software.amazon.awssdk.services.dynamodb.model.WriteRequest + +/** + * INTERNAL API + */ +@InternalApi private[projection] object OffsetStoreDao { + private val log: Logger = LoggerFactory.getLogger(classOf[OffsetStoreDao]) + + // Hard limits in DynamoDB + private val MaxBatchSize = 25 + private val MaxTransactItems = 100 + + object OffsetStoreAttributes { + // FIXME should attribute names be shorter? + val Pid = "pid" + val SeqNr = "seq_nr" + val NameSlice = "name_slice" + val Timestamp = "ts" + val Seen = "seen" + val Paused = "paused" + val Expiry = "expiry" + + val timestampBySlicePid = AttributeValue.fromS("_") + val managementStateBySlicePid = AttributeValue.fromS("_mgmt") + } +} + +/** + * INTERNAL API + */ +@InternalApi private[projection] class OffsetStoreDao( + system: ActorSystem[_], + settings: DynamoDBProjectionSettings, + projectionId: ProjectionId, + client: DynamoDbAsyncClient) { + import OffsetStoreDao.log + import OffsetStoreDao.MaxBatchSize + import OffsetStoreDao.MaxTransactItems + import system.executionContext + + private val timeToLiveSettings = settings.timeToLiveSettings.projections.get(projectionId.name) + + private def nameSlice(slice: Int): String = s"${projectionId.name}-$slice" + + def loadTimestampOffset(slice: Int): Future[Option[TimestampOffset]] = { + import OffsetStoreDao.OffsetStoreAttributes._ + val expressionAttributeValues = + Map(":nameSlice" -> AttributeValue.fromS(nameSlice(slice)), ":pid" -> timestampBySlicePid).asJava + + val req = QueryRequest.builder + .tableName(settings.timestampOffsetTable) + .consistentRead(false) // not necessary to read latest, can start at earlier time + .keyConditionExpression(s"$NameSlice = :nameSlice AND $Pid = :pid") + .expressionAttributeValues(expressionAttributeValues) + .projectionExpression(s"$Timestamp, $Seen") + .build() + + client + .query(req) + .asScala + .map { response => + val items = response.items() + if (items.isEmpty) + None + else { + val item = items.get(0) + val timestampMicros = item.get(Timestamp).n().toLong + val timestamp = InstantFactory.fromEpochMicros(timestampMicros) + val seen = item.get(Seen).m().asScala.iterator.map { case (pid, attr) => pid -> attr.n().toLong }.toMap + val timestampOffset = TimestampOffset(timestamp, seen) + Some(timestampOffset) + } + } + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + def storeTimestampOffsets(offsetsBySlice: Map[Int, TimestampOffset]): Future[Done] = { + import OffsetStoreDao.OffsetStoreAttributes._ + + val expiry = timeToLiveSettings.offsetTimeToLive.map { timeToLive => + Instant.now().plusSeconds(timeToLive.toSeconds) + } + + def writeBatch(offsetsBatch: IndexedSeq[(Int, TimestampOffset)]): Future[Done] = { + val writeItems = + offsetsBatch.map { + case (slice, offset) => + val attributes = new JHashMap[String, AttributeValue] + attributes.put(NameSlice, AttributeValue.fromS(nameSlice(slice))) + attributes.put(Pid, timestampBySlicePid) + val timestampMicros = InstantFactory.toEpochMicros(offset.timestamp) + attributes.put(Timestamp, AttributeValue.fromN(timestampMicros.toString)) + val seen = { + if (offset.seen.isEmpty) + Collections.emptyMap[String, AttributeValue] + else if (offset.seen.size == 1) + Collections.singletonMap(offset.seen.head._1, AttributeValue.fromN(offset.seen.head._2.toString)) + else { + val seen = new JHashMap[String, AttributeValue] + offset.seen.iterator.foreach { + case (pid, seqNr) => seen.put(pid, AttributeValue.fromN(seqNr.toString)) + } + seen + } + } + attributes.put(Seen, AttributeValue.fromM(seen)) + + expiry.foreach { timestamp => + attributes.put(Expiry, AttributeValue.fromN(timestamp.getEpochSecond.toString)) + } + + WriteRequest.builder + .putRequest( + PutRequest + .builder() + .item(attributes) + .build()) + .build() + }.asJava + + val req = BatchWriteItemRequest + .builder() + .requestItems(Collections.singletonMap(settings.timestampOffsetTable, writeItems)) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL) + .build() + + val result = client.batchWriteItem(req).asScala + + if (log.isDebugEnabled()) { + result.foreach { response => + log.debug( + "Wrote latest timestamps for [{}] slices, consumed [{}] WCU", + offsetsBatch.size, + response.consumedCapacity.iterator.asScala.map(_.capacityUnits.doubleValue()).sum) + } + } + result + .map(_ => Done)(ExecutionContext.parasitic) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + if (offsetsBySlice.size <= MaxBatchSize) { + writeBatch(offsetsBySlice.toVector) + } else { + val batches = offsetsBySlice.toVector.sliding(MaxBatchSize, MaxBatchSize) + Future + .sequence(batches.map(writeBatch)) + .map(_ => Done)(ExecutionContext.parasitic) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + } + + def storeSequenceNumbers(records: IndexedSeq[Record]): Future[Done] = { + val expiry = timeToLiveSettings.offsetTimeToLive.map { timeToLive => + Instant.now().plusSeconds(timeToLive.toSeconds) + } + + def writeBatch(recordsBatch: IndexedSeq[Record]): Future[Done] = { + val writeItems = + recordsBatch + .map { record => + WriteRequest.builder + .putRequest( + PutRequest + .builder() + .item(sequenceNumberAttributes(record, expiry)) + .build()) + .build() + } + .toVector + .asJava + + val req = BatchWriteItemRequest + .builder() + .requestItems(Collections.singletonMap(settings.timestampOffsetTable, writeItems)) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL) + .build() + + val result = client.batchWriteItem(req).asScala + + if (log.isDebugEnabled()) { + result.foreach { response => + log.debug( + "Wrote [{}] sequence numbers, consumed [{}] WCU", + recordsBatch.size, + response.consumedCapacity.iterator.asScala.map(_.capacityUnits.doubleValue()).sum) + } + } + + result.map(_ => Done)(ExecutionContext.parasitic) + } + + if (records.size <= MaxBatchSize) { + writeBatch(records) + } else { + val batches = records.sliding(MaxBatchSize, MaxBatchSize) + Future + .sequence(batches.map(writeBatch)) + .map(_ => Done)(ExecutionContext.parasitic) + } + } + + def loadSequenceNumber(slice: Int, pid: String): Future[Option[Record]] = { + import OffsetStoreDao.OffsetStoreAttributes._ + val expressionAttributeValues = + Map(":nameSlice" -> AttributeValue.fromS(nameSlice(slice)), ":pid" -> AttributeValue.fromS(pid)).asJava + + val req = QueryRequest.builder + .tableName(settings.timestampOffsetTable) + .consistentRead(true) + .keyConditionExpression(s"$NameSlice = :nameSlice AND $Pid = :pid") + .expressionAttributeValues(expressionAttributeValues) + .projectionExpression(s"$SeqNr, $Timestamp") + .build() + + client + .query(req) + .asScala + .map { response => + val items = response.items() + if (items.isEmpty) + None + else { + val item = items.get(0) + val seqNr = item.get(SeqNr).n().toLong + val timestampMicros = item.get(Timestamp).n().toLong + val timestamp = InstantFactory.fromEpochMicros(timestampMicros) + Some(DynamoDBOffsetStore.Record(slice, pid, seqNr, timestamp)) + } + } + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + def transactStoreSequenceNumbers(writeItems: Iterable[TransactWriteItem])(records: Seq[Record]): Future[Done] = { + if ((writeItems.size + records.size) > MaxTransactItems) + throw new IllegalArgumentException( + s"Too many transactional write items. Total limit is [${MaxTransactItems}], attempting to store " + + s"[${writeItems.size}] write items and [${records.size}] sequence numbers.") + + val expiry = timeToLiveSettings.offsetTimeToLive.map { timeToLive => + Instant.now().plusSeconds(timeToLive.toSeconds) + } + + val writeSequenceNumbers = records.map { record => + TransactWriteItem.builder + .put( + Put + .builder() + .tableName(settings.timestampOffsetTable) + .item(sequenceNumberAttributes(record, expiry)) + .build()) + .build() + } + + val allTransactItems = (writeItems ++ writeSequenceNumbers).asJavaCollection + + val request = TransactWriteItemsRequest + .builder() + .transactItems(allTransactItems) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL) + .build() + + val result = client.transactWriteItems(request).asScala + + if (log.isDebugEnabled()) { + result.foreach { response => + log.debug( + "Atomically wrote [{}] items with [{}] sequence numbers, consumed [{}] WCU", + writeItems.size, + writeSequenceNumbers.size, + response.consumedCapacity.iterator.asScala.map(_.capacityUnits.doubleValue()).sum) + } + } + + result + .map(_ => Done)(ExecutionContext.parasitic) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + + } + + private def sequenceNumberAttributes(record: Record, expiry: Option[Instant]): JHashMap[String, AttributeValue] = { + import OffsetStoreDao.OffsetStoreAttributes._ + + val attributes = new JHashMap[String, AttributeValue] + attributes.put(NameSlice, AttributeValue.fromS(nameSlice(record.slice))) + attributes.put(Pid, AttributeValue.fromS(record.pid)) + attributes.put(SeqNr, AttributeValue.fromN(record.seqNr.toString)) + val timestampMicros = InstantFactory.toEpochMicros(record.timestamp) + attributes.put(Timestamp, AttributeValue.fromN(timestampMicros.toString)) + + expiry.foreach { expiryTimestamp => + attributes.put(Expiry, AttributeValue.fromN(expiryTimestamp.getEpochSecond.toString)) + } + + attributes + } + + def readManagementState(slice: Int): Future[Option[ManagementState]] = { + import OffsetStoreDao.OffsetStoreAttributes._ + val expressionAttributeValues = + Map(":nameSlice" -> AttributeValue.fromS(nameSlice(slice)), ":pid" -> managementStateBySlicePid).asJava + + val req = QueryRequest.builder + .tableName(settings.timestampOffsetTable) + .consistentRead(true) + .keyConditionExpression(s"$NameSlice = :nameSlice AND $Pid = :pid") + .expressionAttributeValues(expressionAttributeValues) + .projectionExpression(s"$Paused") + .build() + + client + .query(req) + .asScala + .map { response => + val items = response.items() + if (items.isEmpty) + None + else { + val item = items.get(0) + val paused = + if (item.containsKey(Paused)) + item.get(Paused).bool().booleanValue() + else + false + + Some(ManagementState(paused)) + } + } + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + def updateManagementState(minSlice: Int, maxSlice: Int, paused: Boolean): Future[Done] = { + import OffsetStoreDao.OffsetStoreAttributes._ + + def writeBatch(slices: Vector[Int]): Future[Done] = { + val writeItems = + slices.map { slice => + val attributes = new JHashMap[String, AttributeValue] + attributes.put(NameSlice, AttributeValue.fromS(nameSlice(slice))) + attributes.put(Pid, managementStateBySlicePid) + attributes.put(Paused, AttributeValue.fromBool(paused)) + + WriteRequest.builder + .putRequest( + PutRequest + .builder() + .item(attributes) + .build()) + .build() + }.asJava + + val req = BatchWriteItemRequest + .builder() + .requestItems(Collections.singletonMap(settings.timestampOffsetTable, writeItems)) + .returnConsumedCapacity(ReturnConsumedCapacity.TOTAL) + .build() + + val result = client.batchWriteItem(req).asScala + + if (log.isDebugEnabled()) { + result.foreach { response => + log.debug( + "Wrote management state for [{}] slices, consumed [{}] WCU", + slices.size, + response.consumedCapacity.iterator.asScala.map(_.capacityUnits.doubleValue()).sum) + } + } + result + .map(_ => Done)(ExecutionContext.parasitic) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + val sliceRange = (minSlice to maxSlice).toVector + if (sliceRange.size <= MaxBatchSize) { + writeBatch(sliceRange) + } else { + val batches = sliceRange.sliding(MaxBatchSize, MaxBatchSize) + Future + .sequence(batches.map(writeBatch)) + .map(_ => Done)(ExecutionContext.parasitic) + } + } +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/CreateTables.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/CreateTables.scala new file mode 100644 index 000000000..3a0343be9 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/CreateTables.scala @@ -0,0 +1,35 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.javadsl + +import java.util.concurrent.CompletionStage + +import scala.jdk.FutureConverters._ + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.util.TableSettings +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.dynamodb.scaladsl +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient + +object CreateTables { + def createTimestampOffsetStoreTable( + system: ActorSystem[_], + settings: DynamoDBProjectionSettings, + client: DynamoDbAsyncClient, + deleteIfExists: Boolean): CompletionStage[Done] = + createTimestampOffsetStoreTable(system, settings, client, deleteIfExists, TableSettings.Local) + + def createTimestampOffsetStoreTable( + system: ActorSystem[_], + settings: DynamoDBProjectionSettings, + client: DynamoDbAsyncClient, + deleteIfExists: Boolean, + tableSettings: TableSettings): CompletionStage[Done] = + scaladsl.CreateTables + .createTimestampOffsetStoreTable(system, settings, client, deleteIfExists, tableSettings) + .asJava +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBProjection.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBProjection.scala new file mode 100644 index 000000000..3abc8ce51 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBProjection.scala @@ -0,0 +1,168 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.javadsl + +import java.util.Optional +import java.util.function.Supplier + +import scala.jdk.OptionConverters._ + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.annotation.ApiMayChange +import akka.projection.ProjectionContext +import akka.projection.ProjectionId +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.dynamodb.internal.DynamoDBTransactGroupedHandlerAdapter +import akka.projection.dynamodb.internal.DynamoDBTransactHandlerAdapter +import akka.projection.dynamodb.scaladsl +import akka.projection.internal.GroupedHandlerAdapter +import akka.projection.internal.HandlerAdapter +import akka.projection.internal.JavaToScalaBySliceSourceProviderAdapter +import akka.projection.javadsl.AtLeastOnceFlowProjection +import akka.projection.javadsl.AtLeastOnceProjection +import akka.projection.javadsl.ExactlyOnceProjection +import akka.projection.javadsl.GroupedProjection +import akka.projection.javadsl.Handler +import akka.projection.javadsl.SourceProvider +import akka.stream.javadsl.FlowWithContext + +@ApiMayChange +object DynamoDBProjection { + + /** + * Create a [[akka.projection.Projection]] with at-least-once processing semantics. + * + * It stores the offset in a DynamoDB table after the `handler` has processed the envelope. This means that if the + * projection is restarted from previously stored offset then some elements may be processed more than once. + * + * The offset is stored after a time window, or limited by a number of envelopes, whatever happens first. This window + * can be defined with [[akka.projection.javadsl.AtLeastOnceProjection.withSaveOffset]] of the returned + * `AtLeastOnceProjection`. The default settings for the window is defined in configuration section + * `akka.projection.at-least-once`. + */ + def atLeastOnce[Offset, Envelope]( + projectionId: ProjectionId, + settings: Optional[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: Supplier[Handler[Envelope]], + system: ActorSystem[_]): AtLeastOnceProjection[Offset, Envelope] = { + + scaladsl.DynamoDBProjection + .atLeastOnce[Offset, Envelope]( + projectionId, + settings.toScala, + JavaToScalaBySliceSourceProviderAdapter(sourceProvider), + () => HandlerAdapter(handler.get()))(system) + .asInstanceOf[AtLeastOnceProjection[Offset, Envelope]] + } + + /** + * Create a [[akka.projection.Projection]] with exactly-once processing semantics. + * + * The offset is stored in DynamoDB in the same transaction as the `TransactWriteItem`s returned by the `handler`. + */ + def exactlyOnce[Offset, Envelope]( + projectionId: ProjectionId, + settings: Optional[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: Supplier[DynamoDBTransactHandler[Envelope]], + system: ActorSystem[_]): ExactlyOnceProjection[Offset, Envelope] = { + + scaladsl.DynamoDBProjection + .exactlyOnce[Offset, Envelope]( + projectionId, + settings.toScala, + JavaToScalaBySliceSourceProviderAdapter(sourceProvider), + () => new DynamoDBTransactHandlerAdapter(handler.get()))(system) + .asInstanceOf[ExactlyOnceProjection[Offset, Envelope]] + } + + /** + * Create a [[akka.projection.Projection]] that groups envelopes and calls the `handler` with a group of `Envelopes`. + * The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This + * window can be defined with [[akka.projection.javadsl.GroupedProjection.withGroup]] of the returned + * `GroupedProjection`. The default settings for the window is defined in configuration section + * `akka.projection.grouped`. + * + * The offset is stored in DynamoDB in the same transaction as the `TransactWriteItem`s returned by the `handler`. + */ + def exactlyOnceGroupedWithin[Offset, Envelope]( + projectionId: ProjectionId, + settings: Optional[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: Supplier[DynamoDBTransactHandler[java.util.List[Envelope]]], + system: ActorSystem[_]): GroupedProjection[Offset, Envelope] = { + scaladsl.DynamoDBProjection + .exactlyOnceGroupedWithin[Offset, Envelope]( + projectionId, + settings.toScala, + JavaToScalaBySliceSourceProviderAdapter(sourceProvider), + () => new DynamoDBTransactGroupedHandlerAdapter(handler.get()))(system) + .asInstanceOf[GroupedProjection[Offset, Envelope]] + } + + /** + * Create a [[akka.projection.Projection]] that groups envelopes and calls the `handler` with a group of `Envelopes`. + * The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This + * window can be defined with [[akka.projection.javadsl.GroupedProjection.withGroup]] of the returned + * `GroupedProjection`. The default settings for the window is defined in configuration section + * `akka.projection.grouped`. + * + * The offset is stored in DynamoDB immediately after the `handler` has processed the envelopes, but that is still + * with at-least-once processing semantics. This means that if the projection is restarted from previously stored + * offset the previous group of envelopes may be processed more than once. + */ + def atLeastOnceGroupedWithin[Offset, Envelope]( + projectionId: ProjectionId, + settings: Optional[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: Supplier[Handler[java.util.List[Envelope]]], + system: ActorSystem[_]): GroupedProjection[Offset, Envelope] = { + scaladsl.DynamoDBProjection + .atLeastOnceGroupedWithin[Offset, Envelope]( + projectionId, + settings.toScala, + JavaToScalaBySliceSourceProviderAdapter(sourceProvider), + () => new GroupedHandlerAdapter(handler.get()))(system) + .asInstanceOf[GroupedProjection[Offset, Envelope]] + } + + /** + * Create a [[akka.projection.Projection]] with a [[akka.stream.javadsl.FlowWithContext]] as the envelope handler. It + * has at-least-once processing semantics. + * + * The flow should emit a `Done` element for each completed envelope. The offset of the envelope is carried in the + * context of the `FlowWithContext` and is stored in the database when corresponding `Done` is emitted. Since the + * offset is stored after processing the envelope it means that if the projection is restarted from previously stored + * offset then some envelopes may be processed more than once. + * + * If the flow filters out envelopes the corresponding offset will not be stored, and such envelope will be processed + * again if the projection is restarted and no later offset was stored. + * + * The flow should not duplicate emitted envelopes (`mapConcat`) with same offset, because then it can result in that + * the first offset is stored and when the projection is restarted that offset is considered completed even though + * more of the duplicated envelopes were never processed. + * + * The flow must not reorder elements, because the offsets may be stored in the wrong order and when the projection is + * restarted all envelopes up to the latest stored offset are considered completed even though some of them may not + * have been processed. This is the reason the flow is restricted to `FlowWithContext` rather than ordinary `Flow`. + */ + def atLeastOnceFlow[Offset, Envelope]( + projectionId: ProjectionId, + settings: Optional[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: FlowWithContext[Envelope, ProjectionContext, Done, ProjectionContext, _], + system: ActorSystem[_]): AtLeastOnceFlowProjection[Offset, Envelope] = { + scaladsl.DynamoDBProjection + .atLeastOnceFlow[Offset, Envelope]( + projectionId, + settings.toScala, + JavaToScalaBySliceSourceProviderAdapter[Offset, Envelope](sourceProvider), + handler.asScala)(system) + .asInstanceOf[AtLeastOnceFlowProjection[Offset, Envelope]] + } + +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBTransactHandler.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBTransactHandler.scala new file mode 100644 index 000000000..27eae20fe --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/javadsl/DynamoDBTransactHandler.scala @@ -0,0 +1,65 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.javadsl + +import java.util.concurrent.CompletableFuture +import java.util.concurrent.CompletionStage +import java.util.function.{ Function => JFunction } +import java.util.{ Collection => JCollection } + +import akka.Done +import akka.annotation.ApiMayChange +import akka.annotation.InternalApi +import akka.projection.javadsl.HandlerLifecycle +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + +/** + * Implement this interface for the Envelope handler for DynamoDB transactional projections. + * + * It can be stateful, with variables and mutable data structures. It is invoked by the `Projection` machinery one + * envelope at a time and visibility guarantees between the invocations are handled automatically, i.e. no volatile or + * other concurrency primitives are needed for managing the state. + * + * Supported error handling strategies for when processing an `Envelope` fails can be defined in configuration or using + * the `withRecoveryStrategy` method of a `Projection` implementation. + */ +@ApiMayChange +trait DynamoDBTransactHandler[Envelope] extends HandlerLifecycle { + + /** + * The `process` method is invoked for each `Envelope`, and should return DynamoDB `TransactWriteItem`s to atomically + * write along with the projection offset. + * + * One envelope is processed at a time. It will not be invoked with the next envelope until the returned + * CompletionStage has completed and the given items written in a transaction. + */ + def process(envelope: Envelope): CompletionStage[JCollection[TransactWriteItem]] + + def start(): CompletionStage[Done] = + CompletableFuture.completedFuture(Done) + + def stop(): CompletionStage[Done] = + CompletableFuture.completedFuture(Done) +} + +@ApiMayChange +object DynamoDBTransactHandler { + + /** + * INTERNAL API + */ + @InternalApi + private class DynamoDBTransactHandlerFunction[Envelope]( + handler: JFunction[Envelope, CompletionStage[JCollection[TransactWriteItem]]]) + extends DynamoDBTransactHandler[Envelope] { + + override def process(envelope: Envelope): CompletionStage[JCollection[TransactWriteItem]] = handler.apply(envelope) + } + + /** DynamoDBTransactHandler that can be defined with a simple function */ + def fromFunction[Envelope](handler: JFunction[Envelope, CompletionStage[JCollection[TransactWriteItem]]]) + : DynamoDBTransactHandler[Envelope] = + new DynamoDBTransactHandlerFunction(handler) +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/CreateTables.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/CreateTables.scala new file mode 100644 index 000000000..d440e8307 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/CreateTables.scala @@ -0,0 +1,108 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.scaladsl + +import java.util.concurrent.CompletionException + +import scala.concurrent.ExecutionContext +import scala.concurrent.Future +import scala.jdk.FutureConverters._ +import scala.util.Failure +import scala.util.Success + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.persistence.dynamodb.util.OnDemandThroughputSettings +import akka.persistence.dynamodb.util.ProvisionedThroughputSettings +import akka.persistence.dynamodb.util.TableSettings +import akka.projection.dynamodb.DynamoDBProjectionSettings +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient +import software.amazon.awssdk.services.dynamodb.model.AttributeDefinition +import software.amazon.awssdk.services.dynamodb.model.CreateTableRequest +import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement +import software.amazon.awssdk.services.dynamodb.model.KeyType +import software.amazon.awssdk.services.dynamodb.model.OnDemandThroughput +import software.amazon.awssdk.services.dynamodb.model.ProvisionedThroughput +import software.amazon.awssdk.services.dynamodb.model.ResourceNotFoundException +import software.amazon.awssdk.services.dynamodb.model.ScalarAttributeType + +object CreateTables { + def createTimestampOffsetStoreTable( + system: ActorSystem[_], + settings: DynamoDBProjectionSettings, + client: DynamoDbAsyncClient, + deleteIfExists: Boolean, + tableSettings: TableSettings = TableSettings.Local): Future[Done] = { + import akka.projection.dynamodb.internal.OffsetStoreDao.OffsetStoreAttributes._ + implicit val ec: ExecutionContext = system.executionContext + + val existingTable = + client.describeTable(DescribeTableRequest.builder().tableName(settings.timestampOffsetTable).build()).asScala + + def create(): Future[Done] = { + var requestBuilder = CreateTableRequest.builder + .tableName(settings.timestampOffsetTable) + .keySchema( + KeySchemaElement.builder().attributeName(NameSlice).keyType(KeyType.HASH).build(), + KeySchemaElement.builder().attributeName(Pid).keyType(KeyType.RANGE).build()) + .attributeDefinitions( + AttributeDefinition.builder().attributeName(NameSlice).attributeType(ScalarAttributeType.S).build(), + AttributeDefinition.builder().attributeName(Pid).attributeType(ScalarAttributeType.S).build()) + + requestBuilder = tableSettings.throughput match { + case provisioned: ProvisionedThroughputSettings => + requestBuilder.provisionedThroughput( + ProvisionedThroughput.builder + .readCapacityUnits(provisioned.readCapacityUnits) + .writeCapacityUnits(provisioned.writeCapacityUnits) + .build()) + case onDemand: OnDemandThroughputSettings => + requestBuilder.onDemandThroughput( + OnDemandThroughput.builder + .maxReadRequestUnits(onDemand.maxReadRequestUnits) + .maxWriteRequestUnits(onDemand.maxWriteRequestUnits) + .build()) + } + + client + .createTable(requestBuilder.build()) + .asScala + .map(_ => Done) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + def delete(): Future[Done] = { + val req = DeleteTableRequest.builder().tableName(settings.timestampOffsetTable).build() + client + .deleteTable(req) + .asScala + .map(_ => Done) + .recoverWith { + case c: CompletionException => + Future.failed(c.getCause) + }(ExecutionContext.parasitic) + } + + existingTable.transformWith { + case Success(_) => + if (deleteIfExists) delete().flatMap(_ => create()) + else Future.successful(Done) + case Failure(_: ResourceNotFoundException) => create() + case Failure(exception: CompletionException) => + exception.getCause match { + case _: ResourceNotFoundException => create() + case cause => Future.failed[Done](cause) + } + case Failure(exc) => + Future.failed[Done](exc) + } + } + +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBProjection.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBProjection.scala new file mode 100644 index 000000000..58948809f --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBProjection.scala @@ -0,0 +1,267 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.scaladsl + +import akka.Done +import akka.actor.typed.ActorSystem +import akka.annotation.ApiMayChange +import akka.persistence.dynamodb.util.ClientProvider +import akka.projection.BySlicesSourceProvider +import akka.projection.ProjectionContext +import akka.projection.ProjectionId +import akka.projection.dynamodb.DynamoDBProjectionSettings +import akka.projection.dynamodb.internal.DynamoDBProjectionImpl +import akka.projection.internal.AtLeastOnce +import akka.projection.internal.ExactlyOnce +import akka.projection.internal.FlowHandlerStrategy +import akka.projection.internal.GroupedHandlerStrategy +import akka.projection.internal.NoopStatusObserver +import akka.projection.internal.OffsetStoredByHandler +import akka.projection.internal.SingleHandlerStrategy +import akka.projection.scaladsl.AtLeastOnceFlowProjection +import akka.projection.scaladsl.AtLeastOnceProjection +import akka.projection.scaladsl.ExactlyOnceProjection +import akka.projection.scaladsl.GroupedProjection +import akka.projection.scaladsl.Handler +import akka.projection.scaladsl.SourceProvider +import akka.stream.scaladsl.FlowWithContext + +@ApiMayChange +object DynamoDBProjection { + + /** + * Create a [[akka.projection.Projection]] with at-least-once processing semantics. + * + * It stores the offset in a DynamoDB table after the `handler` has processed the envelope. This means that if the + * projection is restarted from previously stored offset then some elements may be processed more than once. + * + * The offset is stored after a time window, or limited by a number of envelopes, whatever happens first. This window + * can be defined with [[akka.projection.scaladsl.AtLeastOnceProjection.withSaveOffset]] of the returned + * `AtLeastOnceProjection`. The default settings for the window is defined in configuration section + * `akka.projection.at-least-once`. + */ + def atLeastOnce[Offset, Envelope]( + projectionId: ProjectionId, + settings: Option[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: () => Handler[Envelope])(implicit system: ActorSystem[_]): AtLeastOnceProjection[Offset, Envelope] = { + + val dynamodbSettings = settings.getOrElse(DynamoDBProjectionSettings(system)) + val client = ClientProvider(system).clientFor(dynamodbSettings.useClient) + + val offsetStore = + DynamoDBProjectionImpl.createOffsetStore( + projectionId, + timestampOffsetBySlicesSourceProvider(sourceProvider), + dynamodbSettings, + client) + + val adaptedHandler = + DynamoDBProjectionImpl.adaptedHandlerForAtLeastOnce(sourceProvider, handler, offsetStore)( + system.executionContext, + system) + + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt = None, + sourceProvider, + restartBackoffOpt = None, + offsetStrategy = AtLeastOnce(), + handlerStrategy = SingleHandlerStrategy(adaptedHandler), + NoopStatusObserver, + offsetStore) + } + + /** + * Create a [[akka.projection.Projection]] with exactly-once processing semantics. + * + * The offset is stored in DynamoDB in the same transaction as the `TransactWriteItem`s returned by the `handler`. + */ + def exactlyOnce[Offset, Envelope]( + projectionId: ProjectionId, + settings: Option[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: () => DynamoDBTransactHandler[Envelope])( + implicit + system: ActorSystem[_]): ExactlyOnceProjection[Offset, Envelope] = { + + val dynamodbSettings = settings.getOrElse(DynamoDBProjectionSettings(system)) + val client = ClientProvider(system).clientFor(dynamodbSettings.useClient) + + val offsetStore = + DynamoDBProjectionImpl.createOffsetStore( + projectionId, + timestampOffsetBySlicesSourceProvider(sourceProvider), + dynamodbSettings, + client) + + val adaptedHandler = + DynamoDBProjectionImpl.adaptedHandlerForExactlyOnce(sourceProvider, handler, offsetStore)( + system.executionContext, + system) + + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt = None, + sourceProvider, + restartBackoffOpt = None, + offsetStrategy = ExactlyOnce(), + handlerStrategy = SingleHandlerStrategy(adaptedHandler), + NoopStatusObserver, + offsetStore) + } + + /** + * Create a [[akka.projection.Projection]] that groups envelopes and calls the `handler` with a group of `Envelopes`. + * The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This + * window can be defined with [[akka.projection.scaladsl.GroupedProjection.withGroup]] of the returned + * `GroupedProjection`. The default settings for the window is defined in configuration section + * `akka.projection.grouped`. + * + * The offset is stored in DynamoDB in the same transaction as the `TransactWriteItem`s returned by the `handler`. + */ + def exactlyOnceGroupedWithin[Offset, Envelope]( + projectionId: ProjectionId, + settings: Option[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: () => DynamoDBTransactHandler[Seq[Envelope]])( + implicit + system: ActorSystem[_]): GroupedProjection[Offset, Envelope] = { + + val dynamodbSettings = settings.getOrElse(DynamoDBProjectionSettings(system)) + val client = ClientProvider(system).clientFor(dynamodbSettings.useClient) + + val offsetStore = + DynamoDBProjectionImpl.createOffsetStore( + projectionId, + timestampOffsetBySlicesSourceProvider(sourceProvider), + dynamodbSettings, + client) + + val adaptedHandler = + DynamoDBProjectionImpl.adaptedHandlerForExactlyOnceGrouped(sourceProvider, handler, offsetStore)( + system.executionContext, + system) + + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt = None, + sourceProvider, + restartBackoffOpt = None, + offsetStrategy = ExactlyOnce(), + handlerStrategy = GroupedHandlerStrategy(adaptedHandler), + NoopStatusObserver, + offsetStore) + } + + /** + * Create a [[akka.projection.Projection]] that groups envelopes and calls the `handler` with a group of `Envelopes`. + * The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This + * window can be defined with [[akka.projection.scaladsl.GroupedProjection.withGroup]] of the returned + * `GroupedProjection`. The default settings for the window is defined in configuration section + * `akka.projection.grouped`. + * + * The offset is stored in DynamoDB immediately after the `handler` has processed the envelopes, but that is still + * with at-least-once processing semantics. This means that if the projection is restarted from previously stored + * offset the previous group of envelopes may be processed more than once. + */ + def atLeastOnceGroupedWithin[Offset, Envelope]( + projectionId: ProjectionId, + settings: Option[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: () => Handler[Seq[Envelope]])(implicit system: ActorSystem[_]): GroupedProjection[Offset, Envelope] = { + + val dynamodbSettings = settings.getOrElse(DynamoDBProjectionSettings(system)) + val client = ClientProvider(system).clientFor(dynamodbSettings.useClient) + + val offsetStore = + DynamoDBProjectionImpl.createOffsetStore( + projectionId, + timestampOffsetBySlicesSourceProvider(sourceProvider), + dynamodbSettings, + client) + + val adaptedHandler = + DynamoDBProjectionImpl.adaptedHandlerForAtLeastOnceGrouped(sourceProvider, handler, offsetStore)( + system.executionContext, + system) + + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt = None, + sourceProvider, + restartBackoffOpt = None, + offsetStrategy = OffsetStoredByHandler(), + handlerStrategy = GroupedHandlerStrategy(adaptedHandler), + NoopStatusObserver, + offsetStore) + } + + /** + * Create a [[akka.projection.Projection]] with a [[akka.stream.scaladsl.FlowWithContext]] as the envelope handler. It + * has at-least-once processing semantics. + * + * The flow should emit a `Done` element for each completed envelope. The offset of the envelope is carried in the + * context of the `FlowWithContext` and is stored in the database when corresponding `Done` is emitted. Since the + * offset is stored after processing the envelope it means that if the projection is restarted from previously stored + * offset then some envelopes may be processed more than once. + * + * If the flow filters out envelopes the corresponding offset will not be stored, and such envelope will be processed + * again if the projection is restarted and no later offset was stored. + * + * The flow should not duplicate emitted envelopes (`mapConcat`) with same offset, because then it can result in that + * the first offset is stored and when the projection is restarted that offset is considered completed even though + * more of the duplicated envelopes were never processed. + * + * The flow must not reorder elements, because the offsets may be stored in the wrong order and when the projection is + * restarted all envelopes up to the latest stored offset are considered completed even though some of them may not + * have been processed. This is the reason the flow is restricted to `FlowWithContext` rather than ordinary `Flow`. + */ + def atLeastOnceFlow[Offset, Envelope]( + projectionId: ProjectionId, + settings: Option[DynamoDBProjectionSettings], + sourceProvider: SourceProvider[Offset, Envelope], + handler: FlowWithContext[Envelope, ProjectionContext, Done, ProjectionContext, _])( + implicit + system: ActorSystem[_]): AtLeastOnceFlowProjection[Offset, Envelope] = { + + val dynamodbSettings = settings.getOrElse(DynamoDBProjectionSettings(system)) + val client = ClientProvider(system).clientFor(dynamodbSettings.useClient) + + val offsetStore = + DynamoDBProjectionImpl.createOffsetStore( + projectionId, + timestampOffsetBySlicesSourceProvider(sourceProvider), + dynamodbSettings, + client) + + val adaptedHandler = + DynamoDBProjectionImpl.adaptedHandlerForFlow(sourceProvider, handler, offsetStore, dynamodbSettings)(system) + + new DynamoDBProjectionImpl( + projectionId, + dynamodbSettings, + settingsOpt = None, + sourceProvider, + restartBackoffOpt = None, + offsetStrategy = AtLeastOnce(), + handlerStrategy = FlowHandlerStrategy(adaptedHandler), + NoopStatusObserver, + offsetStore) + } + + private def timestampOffsetBySlicesSourceProvider( + sourceProvider: SourceProvider[_, _]): Option[BySlicesSourceProvider] = { + sourceProvider match { + case provider: BySlicesSourceProvider => Some(provider) + case _ => None // source provider is not using slices + } + } + +} diff --git a/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBTransactHandler.scala b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBTransactHandler.scala new file mode 100644 index 000000000..a01d71863 --- /dev/null +++ b/akka-projection-dynamodb/src/main/scala/akka/projection/dynamodb/scaladsl/DynamoDBTransactHandler.scala @@ -0,0 +1,48 @@ +/* + * Copyright (C) 2024 Lightbend Inc. + */ + +package akka.projection.dynamodb.scaladsl + +import scala.concurrent.Future + +import akka.annotation.ApiMayChange +import akka.projection.scaladsl.HandlerLifecycle +import software.amazon.awssdk.services.dynamodb.model.TransactWriteItem + +/** + * Implement this interface for the Envelope handler for DynamoDB transactional projections. + * + * It can be stateful, with variables and mutable data structures. It is invoked by the `Projection` machinery one + * envelope at a time and visibility guarantees between the invocations are handled automatically, i.e. no volatile or + * other concurrency primitives are needed for managing the state. + * + * Supported error handling strategies for when processing an `Envelope` fails can be defined in configuration or using + * the `withRecoveryStrategy` method of a `Projection` implementation. + */ +@ApiMayChange +trait DynamoDBTransactHandler[Envelope] extends HandlerLifecycle { + + /** + * The `process` method is invoked for each `Envelope`, and should return DynamoDB `TransactWriteItem`s to atomically + * write along with the projection offset. + * + * One envelope is processed at a time. It will not be invoked with the next envelope until the returned Future has + * completed and the given items written in a transaction. + */ + def process(envelope: Envelope): Future[Iterable[TransactWriteItem]] +} + +@ApiMayChange +object DynamoDBTransactHandler { + + private class DynamoDBTransactHandlerFunction[Envelope](handler: Envelope => Future[Iterable[TransactWriteItem]]) + extends DynamoDBTransactHandler[Envelope] { + + override def process(envelope: Envelope): Future[Iterable[TransactWriteItem]] = handler(envelope) + } + + /** DynamoDBTransactHandler that can be defined with a simple function */ + def apply[Envelope](handler: Envelope => Future[Iterable[TransactWriteItem]]): DynamoDBTransactHandler[Envelope] = + new DynamoDBTransactHandlerFunction(handler) +} diff --git a/build.sbt b/build.sbt index f8f5f2679..ca8a3b605 100644 --- a/build.sbt +++ b/build.sbt @@ -37,7 +37,7 @@ lazy val jdbc = .settings(Dependencies.jdbc) .settings( // needed because slick pulls in 2.2.0 - dependencyOverrides += Dependencies.Compile.sl4j) + dependencyOverrides += Dependencies.Compile.slf4j) .dependsOn(core) .dependsOn(coreTest % "test->test") .dependsOn(testkit % Test) @@ -58,7 +58,7 @@ lazy val slick = .settings(Dependencies.slick) .settings( // needed because slick pulls in 2.2.0 - dependencyOverrides += Dependencies.Compile.sl4j) + dependencyOverrides += Dependencies.Compile.slf4j) .dependsOn(jdbc, core) .disablePlugins(CiReleasePlugin) @@ -68,7 +68,7 @@ lazy val slickIntegration = .settings(Dependencies.slickIntegration) .settings( // needed because slick pulls in 2.2.0 - dependencyOverrides += Dependencies.Compile.sl4j) + dependencyOverrides += Dependencies.Compile.slf4j) .dependsOn(slick) .dependsOn(coreTest % "test->test") .dependsOn(testkit % Test) @@ -110,7 +110,7 @@ lazy val kafkaIntegration = .settings(Dependencies.kafkaIntegration) .settings( // needed because test uses Slick which pulls in 2.2.0 - dependencyOverrides += Dependencies.Compile.sl4j) + dependencyOverrides += Dependencies.Compile.slf4j) .dependsOn(kafka, testkit % "test->test") .dependsOn(slick) .dependsOn(slickIntegration % "test->test") @@ -171,6 +171,22 @@ lazy val r2dbcIntegration = .dependsOn(testkit % Test) .disablePlugins(CiReleasePlugin) +lazy val dynamodb = + Project(id = "akka-projection-dynamodb", base = file("akka-projection-dynamodb")) + .settings(Dependencies.dynamodb) + .dependsOn(core, eventsourced) + .disablePlugins(CiReleasePlugin) + // FIXME: No previous artifact, disable MiMa until first release + .settings(mimaPreviousArtifacts := Set.empty) + +lazy val dynamodbIntegration = + Project(id = "akka-projection-dynamodb-integration", base = file("akka-projection-dynamodb-integration")) + .settings(IntegrationTests.settings) + .settings(Dependencies.dynamodbIntegration) + .dependsOn(dynamodb) + .dependsOn(testkit % Test) + .disablePlugins(CiReleasePlugin) + // note that this is in the integration test aggregate // rather than root since it depends on other integration test modules lazy val examples = project @@ -208,6 +224,8 @@ lazy val commonParadoxProperties = Def.settings( "javadoc.akka.grpc.base_url" -> "", // Akka persistence R2DBC plugin "extref.akka-persistence-r2dbc.base_url" -> s"https://doc.akka.io/libraries/akka-persistence-r2dbc/${Dependencies.Versions.AkkaPersistenceR2dbcVersionInDocs}/%s", + // Akka Persistence DynamoDB plugin + "extref.akka-persistence-dynamodb.base_url" -> s"https://doc.akka.io/libraries/akka-persistence-dynamodb/${Dependencies.Versions.AkkaPersistenceDynamodbVersionInDocs}/%s", // Akka Guide "extref.akka-guide.base_url" -> "https://developer.lightbend.com/docs/akka-guide/microservices-tutorial/", // Java @@ -323,6 +341,7 @@ lazy val root = Project(id = "akka-projection", base = file(".")) grpc, grpcTests, r2dbc, + dynamodb, docs, `akka-distributed-cluster-docs`, `akka-edge-docs`) @@ -339,6 +358,7 @@ lazy val integrationTests = Project(id = "akka-projection-integration", base = f jdbcIntegration, kafkaIntegration, r2dbcIntegration, + dynamodbIntegration, slickIntegration) .settings(publish / skip := true) .disablePlugins(SitePlugin, MimaPlugin, CiReleasePlugin) diff --git a/docker-files/docker-compose-dynamodb.yml b/docker-files/docker-compose-dynamodb.yml new file mode 100644 index 000000000..e191c513d --- /dev/null +++ b/docker-files/docker-compose-dynamodb.yml @@ -0,0 +1,8 @@ +services: + dynamodb-local: + command: "-jar DynamoDBLocal.jar -sharedDb -inMemory" + image: "amazon/dynamodb-local:latest" + container_name: dynamodb-local + ports: + - "8000:8000" + working_dir: /home/dynamodblocal diff --git a/docs/src/main/paradox/dynamodb.md b/docs/src/main/paradox/dynamodb.md new file mode 100644 index 000000000..4439acec2 --- /dev/null +++ b/docs/src/main/paradox/dynamodb.md @@ -0,0 +1,346 @@ +# Offset in DynamoDB + +@apidoc[DynamoDBProjection$] has support for storing offsets in DynamoDB via @extref:[Akka Persistence DynamoDB](akka-persistence-dynamodb:overview.html). + +The source of the envelopes is from a `SourceProvider`, which can be: + +* events from Event Sourced entities via the @ref:[SourceProvider for eventsBySlices](eventsourced.md#sourceprovider-for-eventsbyslices) with the @extref:[eventsBySlices query](akka-persistence-dynamodb:query.html#eventsbyslices) + +The target database operations can run in the same transaction as storing the offset, so that +@ref:[exactly-once](#exactly-once) processing semantics are supported. It also offers +@ref:[at-least-once](#at-least-once) semantics. + +## Dependencies + +The Akka dependencies are available from Akka's library repository. To access them there, you need to configure the URL for this repository. + +@@repository [Maven,sbt,Gradle] { +id="akka-repository" +name="Akka library repository" +url="https://repo.akka.io/maven" +} + +To use the DynamoDB module of Akka Projections, add the following dependencies to your project: + +@@dependency [Maven,sbt,Gradle] { +group=com.lightbend.akka +artifact=akka-projection-dynamodb_$scala.binary.version$ +version=$project.version$ +group2=com.lightbend.akka +artifact2=akka-persistence-dynamodb_$scala.binary.version$ +version2=$project.version$ +} + +Akka Projection DynamoDB depends on Akka $akka.version$ or later, and note that it is important that all `akka-*` +dependencies are in the same version, so it is recommended to depend on them explicitly to avoid problems with +transient dependencies causing an unlucky mix of versions. + +@@project-info{ projectId="akka-projection-dynamodb" } + +### Transitive dependencies + +The table below shows `akka-projection-dynamodb`'s direct dependencies, and the second tab shows all libraries it depends on transitively. + +@@dependencies{ projectId="akka-projection-dynamodb" } + +## Tables + +Akka Projection DynamoDB requires an offset table to be created in DynamoDB. The default table name is +`timestamp_offset` and this can be configured (see the @ref:[reference configuration](#reference-configuration) for all +settings). The table should be created with the following attributes and key schema: + +| Attribute name | Attribute type | Key type | +| -------------- | -------------- | -------- | +| name_slice | S (String) | HASH | +| pid | S (String) | RANGE | + +Read and write capacity units should be based on expected projection activity. + +An example `aws` CLI command for creating the timestamp offset table: + +@@snip [aws create timestamp offset table](/akka-projection-dynamodb/scripts/create-tables.sh) { #create-timestamp-offset-table } + +### Creating tables locally + +The DynamoDB client @extref:[can be configured](akka-persistence-dynamodb:config.html#dynamodb-client-configuration) +with a local mode, for testing with DynamoDB local: + +@@snip [local mode](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #local-mode } + +Similar to @extref:[creating tables locally](akka-persistence-dynamodb:getting-started.html#creating-tables-locally) +for Akka Persistence DynamoDB, a @apidoc[akka.projection.dynamodb.*.CreateTables$] utility is provided for creating +projection tables locally: + +Java +: @@snip [create tables](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #create-tables } + +Scala +: @@snip [create tables](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #create-tables } + +## Configuration + +By default, Akka Projection DynamoDB shares the @extref:[DynamoDB client configuration](akka-persistence-dynamodb:config.html#dynamodb-client-configuration) with Akka Persistence DynamoDB. + +### Reference configuration + +The following can be overridden in your `application.conf` for projection specific settings: + +@@snip [reference.conf](/akka-projection-dynamodb/src/main/resources/reference.conf) { #projection-config } + +## Running with Sharded Daemon Process + +The Sharded Daemon Process can be used to distribute `n` instances of a given Projection across the cluster. +Therefore, it's important that each Projection instance consumes a subset of the stream of envelopes. + +When using `eventsBySlices` the initialization code looks like this: + +Java +: @@snip [init projections](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #init-projections } + +Scala +: @@snip [init projections](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #init-projections } + +The @ref:[`ShoppingCartTransactHandler` is shown below](#transact-handler). + +It is possible to dynamically scale the number of Projection instances as described in the @extref:[Sharded Daemon +Process documentation](akka:typed/cluster-sharded-daemon-process.html#dynamic-scaling-of-number-of-workers). + +There are alternative ways of running the `ProjectionBehavior` as described in @ref:[Running a Projection](running.md). + +## Slices + +The `SourceProvider` for Event Sourced actors has historically been using `eventsByTag` but the DynamoDB plugin is +instead providing `eventsBySlices` as an improved solution. + +The usage of `eventsByTag` for Projections has the drawback that the number of tags must be decided up-front and can't +easily be changed afterwards. Starting with too many tags means a lot of overhead, since many projection instances +would be running on each node in a small Akka Cluster, with each projection instance polling the database periodically. +Starting with too few tags means that it can't be scaled later to more Akka nodes. + +With `eventsBySlices` more Projection instances can be added when needed and still reuse the offsets for the previous +slice distributions. + +A slice is deterministically defined based on the persistence id. The purpose is to evenly distribute all +persistence ids over the slices. The `eventsBySlices` query is for a range of the slices. For example if +using 1024 slices and running 4 Projection instances the slice ranges would be 0-255, 256-511, 512-767, 768-1023. +Changing to 8 slice ranges means that the ranges would be 0-127, 128-255, 256-383, ..., 768-895, 896-1023. + +However, when changing the number of slices the projections with the old slice distribution must be stopped before +starting new projections. That can be done at runtime when @ref:[Running with Sharded Daemon +Process](#running-with-sharded-daemon-process). + +When using `DynamoDBProjection` together with the `EventSourcedProvider.eventsBySlices` the events will be delivered in +sequence number order without duplicates. + +## exactly-once + +The offset is stored in the same transaction as items returned by the projection handler, providing exactly-once +processing semantics if the projection is restarted from a previously stored offset. A @apidoc[DynamoDBTransactHandler] +is implemented, returning a collection of DynamoDB `TransactWriteItem`s which will be stored in the same transaction. + +Java +: @@snip [exactly once](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #projection-imports #exactly-once } + +Scala +: @@snip [exactly once](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #exactly-once } + +The @ref:[`ShoppingCartTransactHandler` is shown below](#transact-handler). + +## at-least-once + +The offset is stored after the envelope has been processed, providing at-least-once processing semantics. This means +that if the projection is restarted from a previously stored offset some elements may be processed more than once. +Therefore, the @ref:[Handler](#handler) code must be idempotent. + +Java +: @@snip [at least once](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #projection-imports #at-least-once } + +Scala +: @@snip [at least once](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #at-least-once } + +The offset is stored after a time window, or limited by a number of envelopes, whatever happens first. This window can +be defined with `withSaveOffset` of the returned `AtLeastOnceProjection`. The default settings for the window is +defined in the configuration section `akka.projection.at-least-once`. There is a performance benefit of not storing the +offset too often, but the drawback is that there can be more duplicates that will be processed again when the +projection is restarted. + +The @ref:[`ShoppingCartHandler` is shown below](#generic-handler). + +## exactly-once (grouped) + +The envelopes can be grouped before processing, which can be useful for batch updates. + +The offset is stored in the same transaction as items returned by the projection handler, providing exactly-once +processing semantics if the projection is restarted from a previously stored offset. A @apidoc[DynamoDBTransactHandler] +is implemented, returning a collection of DynamoDB `TransactWriteItem`s which will be stored in the same transaction. + +The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This +window can be defined using `withGroup` of the returned `GroupedProjection`. The default settings for the window is +defined in the configuration section `akka.projection.grouped`. + +Java +: @@snip [exactly once grouped within](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #projection-imports #exactly-once-grouped-within } + +Scala +: @@snip [exactly once grouped within](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #exactly-once-grouped-within } + +The @ref:[`GroupedShoppingCartTransactHandler` is shown below](#grouped-transact-handler). + +## at-least-once (grouped) + +The envelopes can be grouped before processing, which can be useful for batch updates. + +The offsets are stored after the envelopes have been processed, providing at-least-once processing semantics. This +means that if the projection is restarted from a previously stored offset some elements may be processed more than +once. Therefore, the @ref:[Handler](#handler) code must be idempotent. + +The envelopes are grouped within a time window, or limited by a number of envelopes, whatever happens first. This +window can be defined using `withGroup` of the returned `GroupedProjection`. The default settings for the window is +defined in the configuration section `akka.projection.grouped`. + +Java +: @@snip [at least once grouped within](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #projection-imports #at-least-once-grouped-within } + +Scala +: @@snip [at least once grouped within](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #at-least-once-grouped-within } + +The @ref:[`GroupedShoppingCartHandler` is shown below](#grouped-handler). + +## Handler + +For at-least-once processing, a generic projection @apidoc[Handler] is implemented and projections can do any +processing, with or without DynamoDB. + +For exactly-once processing, a @apidoc[DynamoDBTransactHandler] is implemented, returning a collection of DynamoDB +`TransactWriteItem`s which will be written in the same transaction as storing the offsets. + +### Generic handler + +A generic @apidoc[Handler] that is consuming `ShoppingCart.Event` from `eventsBySlices` can look like this: + +Java +: @@snip [handler](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #handler } + +Scala +: @@snip [handler](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #handler } + +@@@ note { title=Hint } +Simple handlers can also be defined as plain functions via the helper @scala[`Handler.apply`]@java[`Handler.fromFunction`] factory method. +@@@ + +### Transact handler + +A @apidoc[DynamoDBTransactHandler] that is consuming `ShoppingCart.Event` from `eventsBySlices` can look like this: + +Java +: @@snip [transact handler](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #transact-handler } + +Scala +: @@snip [transact handler](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #transact-handler } + +@@@ note { title=Hint } +Simple handlers can also be defined as plain functions via the helper @scala[`DynamoDBTransactHandler.apply`]@java[`DynamoDBTransactHandler.fromFunction`] factory method. +@@@ + +### Grouped handler + +When using @ref:[`DynamoDBProjection.atLeastOnceGroupedWithin`](#at-least-once-grouped-) the handler is processing a @scala[`Seq`]@java[`List`] of envelopes. + +Java +: @@snip [grouped handler](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #grouped-handler } + +Scala +: @@snip [grouped handler](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #grouped-handler } + +### Grouped transact handler + +When using @ref:[`DynamoDBProjection.exactlyOnceGroupedWithin`](#exactly-once-grouped-) the +@apidoc[DynamoDBTransactHandler] is processing a @scala[`Seq`]@java[`List`] of envelopes. + +Java +: @@snip [grouped handler](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #grouped-transact-handler } + +Scala +: @@snip [grouped handler](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #grouped-transact-handler } + +### Stateful handler + +The @apidoc[Handler] or @apidoc[DynamoDBTransactHandler] can be stateful, with variables and mutable data structures. +It is invoked by the `Projection` machinery one envelope or group of envelopes at a time and visibility guarantees +between the invocations are handled automatically, i.e. no volatile or other concurrency primitives are needed for +managing the state as long as it's not accessed by other threads than the one that called `process`. + +@@@ note +It is important that the handler instance is not shared between several projection instances, because then it would be +invoked concurrently, which is not how it is intended to be used. Each projection instance should use a new handler +instance. +@@@ + +### Actor handler + +A good alternative for advanced state management is to implement the handler as an +@extref:[actor](akka:typed/actors.html) which is described in @ref:[Processing with Actor](actor.md). + +### Flow handler + +An Akka Streams `FlowWithContext` can be used instead of a handler for processing the envelopes, which is described in +@ref:[Processing with Akka Streams](flow.md). + +### Handler lifecycle + +You can override the `start` and `stop` methods of the @apidoc[Handler] or @apidoc[DynamoDBTransactHandler] to +implement initialization before the first envelope is processed and resource cleanup when the projection is stopped. +Those methods are also called when the projection is restarted after failure. + +See also @ref:[error handling](error.md). + +## Publish events for lower latency + +See @extref:[eventsBySlices documentation](akka-persistence-dynamodb:query.html#publish-events-for-lower-latency-of-eventsbyslices). + +## Multiple plugins + +Similar to how multiple plugins can be configured for the @extref[DynamoDB persistence +plugin](akka-persistence-dynamodb:config.html#multiple-plugins), multiple projection configurations are also possible. + +For the projection offset store you need an additional config section: + +@@snip [second config](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #second-projection-config } + +Note that the `use-client` property references the same client settings as used for the `second-dynamodb` plugins, but +it could also have been a separate client configured as: + +@@snip [second config with client](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #second-projection-config-with-client } + +In this way, you can use the default plugins for the write side and projection `SourceProvider`, but use a separate configuration for the projection handlers and offset storage. + +You start the projections with @apidoc[DynamoDBProjectionSettings] loaded from `"second-projection-dynamodb"`. + +Java +: @@snip [projection settings](/akka-projection-dynamodb-integration/src/test/java/projection/docs/javadsl/ProjectionDocExample.java) { #projection-settings } + +Scala +: @@snip [projection settings](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/scaladsl/ProjectionDocExample.scala) { #projection-settings } + +## Time to Live (TTL) + +Offsets are never deleted by default. To have offsets deleted for inactive projections, an expiration timestamp can be +set. DynamoDB's [Time to Live (TTL)][ttl] feature can then be enabled, to automatically delete items after they have +expired. A new expiration timestamp will be set each time an offset for a particular projection slice or persistence id +is updated. + +The TTL attribute to use for the timestamp offset table is named `expiry`. + +Time-to-live settings are configured per projection. The projection name can also be matched by prefix by using a `*` +at the end of the key. For example, offsets can be configured to expire in 7 days for a particular projection, and in +14 days for all projection names that start with a particular prefix: + +@@ snip [offset time-to-live](/akka-projection-dynamodb-integration/src/test/scala/projection/docs/config/ProjectionTimeToLiveSettingsDocExample.scala) { #time-to-live type=conf } + +### Time to Live reference configuration + +The following can be overridden in your `application.conf` for the time-to-live specific settings: + +@@snip [reference.conf](/akka-projection-dynamodb/src/main/resources/reference.conf) { #time-to-live-settings } + +[ttl]: https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/TTL.html diff --git a/docs/src/main/paradox/index.md b/docs/src/main/paradox/index.md index 861f657a0..41b420c8b 100644 --- a/docs/src/main/paradox/index.md +++ b/docs/src/main/paradox/index.md @@ -16,6 +16,7 @@ * [Cassandra](cassandra.md) * [JDBC](jdbc.md) * [Slick](slick.md) +* [DynamoDB](dynamodb.md) * [Running](running.md) * [Actor](actor.md) * [Flow](flow.md) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 24ff2609b..3d4b82f29 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -38,6 +38,13 @@ object Dependencies { case Seq(major, minor, _*) => s"$major.$minor" } + val AkkaPersistenceDynamodb = "2.0.0-M1" + val AkkaPersistenceDynamodbVersionInDocs = VersionNumber(AkkaPersistenceDynamodb).numbers match { + case Seq(major, minor, _*) => s"$major.$minor" + } + + val DynamodbSdk = "2.25.59" + val alpakkaKafka = sys.props.getOrElse("build.alpakka.kafka.version", "7.0.0-M1") val slick = "3.5.2" val scalaTest = "3.2.18" @@ -70,7 +77,10 @@ object Dependencies { val r2dbcSqlServer = "io.r2dbc" % "r2dbc-mssql" % "1.0.2.RELEASE" % Provided // ApacheV2 - val sl4j = "org.slf4j" % "slf4j-api" % "2.0.16" + val akkaPersistenceDynamodb = "com.lightbend.akka" %% "akka-persistence-dynamodb" % Versions.AkkaPersistenceDynamodb + val dynamodbSdk = "software.amazon.awssdk" % "dynamodb" % Versions.DynamodbSdk + + val slf4j = "org.slf4j" % "slf4j-api" % "2.0.16" val slick = "com.typesafe.slick" %% "slick" % Versions.slick val alpakkaCassandra = "com.lightbend.akka" %% "akka-stream-alpakka-cassandra" % Versions.Alpakka @@ -190,7 +200,7 @@ object Dependencies { val slick = deps ++= Seq( Compile.slick, - Compile.sl4j, + Compile.slf4j, Compile.akkaPersistenceQuery, Test.akkaTypedTestkit, Test.h2Driver, @@ -199,7 +209,7 @@ object Dependencies { val slickIntegration = deps ++= Seq( Compile.slick, - Compile.sl4j, + Compile.slf4j, Compile.akkaPersistenceQuery, Test.akkaTypedTestkit, Test.h2Driver, @@ -306,6 +316,21 @@ object Dependencies { Test.logback, Test.scalatest) + val dynamodb = deps ++= Seq( + Compile.dynamodbSdk.exclude("software.amazon.awssdk", "apache-client"), + Compile.akkaPersistenceDynamodb, + Compile.akkaPersistenceQuery, + Compile.akkaPersistenceTyped, + Compile.akkaStreamTyped) + + val dynamodbIntegration = deps ++= Seq( + Test.akkaStreamTestkit, + Test.akkaTypedTestkit, + Test.akkaClusterShardingTyped, + Test.akkaSerializationJackson, + Test.logback, + Test.scalatest) + val examples = deps ++= Seq( Examples.akkaPersistenceTyped, diff --git a/project/project-info.conf b/project/project-info.conf index db64b83c6..1a40bd0f8 100644 --- a/project/project-info.conf +++ b/project/project-info.conf @@ -174,4 +174,15 @@ project-info { } ] } + akka-projection-dynamodb: ${project-info.shared-info} { + title: "Akka Projection DynamoDB" + jpms-name: "akka.projection.dynamodb" + levels: [ + { + readiness: Supported + since: "2024-10-17" + since-version: "1.6.0" + } + ] + } }