Skip to content

Conversation

@theoema
Copy link

@theoema theoema commented Jan 5, 2026

Hey @smcvb @abuijze @CodeDrivenMitch @schananas! Me and my startup founders are planning on using AF5 in production in the coming months. And one of those features we are going to need is multitenancy, so I've taken it upon myself during the last 3 days to rewrite the multitenancy extension for AF5. Its definitely not perfect, but I've tried my best to keep the extension inline with the spirit of AF5 and I think it went really well considering the time spent.

This is quite a big PR, but I've prepared some documentation both here in the PR and in the project to make it as easy as possible to review. I'm sure reviewing this isnt priority number one for any of your right now, but feel free to let me know if theres anything I could do to make the process easier!


Summary

This PR represents a complete rewrite of the multi-tenancy extension for Axon Framework 5. The extension has been redesigned from the ground up to align with AF5's core principles: async-native architecture, framework-agnostic design (not tied to Spring), and programmatic configuration.

Key Highlights

  • Complete architectural redesign following AF5 patterns
  • New TenantComponent concept for tenant-scoped dependency injection
  • Programmatic configuration via MultiTenancyConfigurer (Spring optional)
  • Full support for both embedded and Axon Server deployments
  • Comprehensive integration test suite
  • Preliminary documentation covering all major use cases

Note: Since I dont have a Axon Server license, I have not been able to test the functionality with multiple contexts on axon server, but it does have tests for embedded multitenancy, and tests for single-tenant axon server setup.

Architectural Changes from Axon 4

Package Structure

  • Renamed from org.axonframework.extensions.multitenancy to org.axonframework.extension.multitenancy (singular) for consistency with AF5 conventions

Configuration Model: From Spring-Centric to Programmatic

Axon 4 relied heavily on Spring XML/YAML configuration and auto-discovery through Spring components.

Axon 5 introduces a layered approach:

  1. Core Layer (multitenancy module) - Pure programmatic configuration, zero Spring dependencies
  2. Spring Layer (multitenancy-spring) - Optional Spring Data JPA support
  3. Spring Boot Layer (multitenancy-spring-boot-autoconfigure) - Auto-configuration for Spring Boot users
// AF5: Programmatic configuration - works without Spring
EventSourcingConfigurer configurer = EventSourcingConfigurer.create();

MultiTenancyConfigurer.enhance(configurer)
    .registerTenantProvider(config -> new SimpleTenantProvider(tenants))
    .registerTargetTenantResolver(config -> new MetadataBasedTenantResolver())
    .tenantComponent(OrderRepository.class, tenant -> new InMemoryOrderRepository());

AxonConfiguration configuration = configurer.start();

Core Abstractions

Component Axon 4 Axon 5
TenantDescriptor Complex with configuration responsibilities Lightweight data holder with tenantId and properties
TenantProvider Static provider Dynamic, subscription-based with subscribe(MultiTenantAwareComponent)
TargetTenantResolver Separate resolvers per message type Unified generic resolver BiFunction<Message, Collection, TenantDescriptor>
Configuration Spring-driven MultiTenancyConfigurer with fluent builder API

TenantComponent: Tenant-Scoped Dependency Injection

A key innovation in this rewrite is the TenantComponent concept, enabling automatic injection of tenant-scoped dependencies into message handlers.

The Problem

In multi-tenant systems, components like repositories need to be isolated per tenant. Previously, developers had to manually resolve tenant context and look up the appropriate instance.

The Solution

// Register a tenant-scoped component
MultiTenancyConfigurer.enhance(configurer)
    .tenantComponent(
        OrderRepository.class,
        tenant -> new JpaOrderRepository(getDataSourceForTenant(tenant))
    );

// Automatically injected with the correct tenant's instance
@EventHandler
void handle(OrderCreatedEvent event, OrderRepository repository) {
    repository.save(new OrderProjection(event));
    // repository is automatically the tenant-specific instance!
}

// Or access via ProcessingContext (TenantAwareProcessingContext)
@EventHandler
void handle(OrderCreatedEvent event, ProcessingContext context) {
    OrderRepository repository = context.component(OrderRepository.class);
    // Same tenant-scoped instance
}

How It Works

  1. Registration: tenantComponent() registers a factory function that creates instances per tenant
  2. Lazy Creation: Instances are created on first access, not on tenant registration
  3. Caching: TenantComponentRegistry caches instances in a ConcurrentHashMap
  4. Resolution: TenantComponentResolver implements ParameterResolver to inject the correct instance
  5. Cleanup: When tenants are removed, AutoCloseable.close() is called automatically

Spring Boot Auto-Discovery

For Spring Boot users, classes implementing TenantComponent are automatically discovered and registered:

  public class TenantAuditService implements TenantComponent<TenantAuditService> {
      private final Clock clock;  // Spring-injected
      private final String tenantId;

      public TenantAuditService(Clock clock) {  // Factory constructor
          this.clock = clock;
          this.tenantId = null;
      }

      @Override
      public TenantAuditService createForTenant(TenantDescriptor tenant) {
          return new TenantAuditService(this.clock, tenant.tenantId());
      }
  }

Spring Data JPA: Automatic Tenant-Scoped Repositories

One of the most powerful features for Spring Boot users is the automatic tenant-scoped JPA repository support. When enabled, Spring Data repositories are automatically discovered, wrapped with per-tenant EntityManagerFactory instances, and injected into handlers with zero configuration. A version of this existed in previous versions of the extension, but it relied on Spring Data's AbstractRoutingDataSource, which is blocking and expects there to be a ThreadLocal context which would be the opposite of async, so I designed a strategy that is async, and I actually think its incredible clean. Now with that said I think that there is a case to be made against per tenant EMFs, which we perhaps should have a discussion about.

Enabling the Feature

axon:
  multi-tenancy:
    tenants:
      - tenant-a
      - tenant-b
    jpa:
      tenant-repositories: true  # This is the key setting

Providing a TenantDataSourceProvider

The only required bean is a TenantDataSourceProvider that returns a DataSource per tenant:

  @Bean
  public TenantDataSourceProvider tenantDataSourceProvider() {
      return tenant -> DataSourceBuilder.create()
              .url("jdbc:postgresql://localhost:5432/" + tenant.tenantId())
              .username("postgres")
              .password("password")
              .build();
  }

Zero-Configuration Repository Usage

Just define your repositories as normal Spring Data interfaces:

  public interface OrderRepository extends JpaRepository<Order, String> {
      List<Order> findByCustomerId(String customerId);
  }

And use them directly in event handlers - the correct tenant's instance is injected automatically:

  @EventHandler
  public void on(OrderCreatedEvent event, OrderRepository repository) {
      // repository is automatically scoped to the tenant from event metadata!
      repository.save(new Order(event.orderId(), event.customerId()));
  }

How It Works Under the Hood

  1. Discovery: MultiTenancySpringDataJpaAutoConfiguration scans for all Repository interfaces
  2. Per-Tenant EMF: TenantEntityManagerFactoryBuilder creates separate EntityManagerFactory per tenant
  3. Transaction Management: TenantTransactionManagerBuilder provides per-tenant transaction managers
  4. Repository Wrapping: TenantJpaRepositoryFactory creates repository instances with:
    - Tenant-specific SharedEntityManager
    - Transactional proxy wrapping all method calls
    - Automatic caching per tenant
  5. Injection: Repositories registered as tenant components, resolved via ParameterResolverFactory

Advanced Configuration

For more control over JPA settings:

  @Bean
  public TenantEntityManagerFactoryBuilder tenantEntityManagerFactoryBuilder(
          TenantDataSourceProvider dataSourceProvider) {
      return TenantEntityManagerFactoryBuilder
              .forDataSourceProvider(dataSourceProvider)
              .packagesToScan("com.myapp.domain")
              .jpaProperty("hibernate.hbm2ddl.auto", "validate")
              .jpaProperty("hibernate.dialect", "org.hibernate.dialect.PostgreSQL10Dialect")
              .persistenceUnitName("myapp")
              .build();
  }

Event Processor Design Considerations

The multi-tenant event processor implementation (MultiTenantPooledStreamingEventProcessorModule) works but required several workarounds due to framework limitations, which sadly led to more than desirable code duplication. I believe this could be significantly cleaner with some extensibility improvements in the main framework.

Suggested Framework Improvements

If we were to make a small change in the main framework, a developer with an pre-existing axon framework codebase wouldn't need to rewrite anything to get the extension working. If we adjust the main framework to register EventProcessor as the interface type instead of the concrete PooledStreamingEventProcessor. This would allow the multi-tenancy extension to use the same decorator pattern as CommandBus, QueryBus, and EventStore - enabling zero-code-change multi-tenancy for event processors. This would mean that framework users can basically "flip a switch" to enable multitenancy. All of their slice configurations would remain the same, they only need to add a few lines of "main configuration" like configuring the tenant resolver, correlation provider and tenantprovider.


Integration Tests

The extension includes comprehensive integration tests covering four deployment scenarios:

Test Modules

Module Description Tests
multitenancy-integration-tests-embedded Pure AF5, in-memory storage 7 tests
multitenancy-integration-tests-axon-server AF5 + Axon Server (Docker) 4 tests
multitenancy-integration-tests-springboot-embedded Spring Boot + JPA (H2) 7 tests
multitenancy-integration-tests-springboot-axonserver Spring Boot + Axon Server (Docker+H2) 4 tests

Scenarios Covered

  • Tenant Isolation: Data from Tenant A not visible in Tenant B
  • Same Entity ID Across Tenants: No conflicts when same ID exists in different tenants
  • Dynamic Tenant Registration: Adding tenants at runtime via SimpleTenantProvider
  • CommandDispatcher Propagation: Tenant context propagates through automation workflows
  • Missing Metadata Handling: Proper NoSuchTenantException when tenant not specified
  • Spring Data Repository Scoping: Automatic tenant-scoped JPA repositories
  • TenantComponent Injection: Correct tenant instance injected into handlers

Test Domain Model

Uses a university course management domain:

  • CreateCourse command → CourseCreated event → CoursesStats projection
  • WhenCourseCreatedThenNotify automation testing CommandDispatcher propagation

JPA-Based Event Store Support

The extension theoretically supports JPA-based event stores through the segment factory pattern:

MultiTenancyConfigurer.enhance(configurer)
    .registerEventStoreSegmentFactory((config, tenant) ->
        createJpaEventStoreForTenant(tenant)
    );

Note: While the architecture supports this, I have not personally tested JPA-based event stores as I'm not familiar with that configuration. The Axon Server and in-memory event store paths are thoroughly tested.


Documentation

Preliminary documentation is available in docs/reference/modules/ROOT/pages/:

Setup Guides (4)

  • Getting Started - Step-by-step transformation from single to multi-tenant
  • Axon Server Integration - Non-Spring applications with Axon Server
  • Spring Boot Integration - Spring Boot with embedded storage
  • Spring Boot + Axon Server - "Golden path" production setup

Concept Documentation (4)

  • Tenant Management - TenantProvider, dynamic tenants
  • Message Routing - TargetTenantResolver, metadata propagation
  • Infrastructure - Multi-tenant buses and event stores
  • Projections - Tenant-scoped repositories and EntityManager

Configuration Reference (4)

  • Tenant Resolution - Custom resolvers
  • Event Processors - MultiTenantPooledStreamingEventProcessorModule
  • Tenant Components - TenantComponent pattern
  • Dynamic Tenants - Runtime tenant management

The documentation covers the major use cases but would benefit from community feedback and real-world usage examples. It's functional but I wouldn't call it polished, it was thrown together quite quickly.


Breaking Changes from Axon 4 Extension

  1. Package rename: extensions.multitenancy → extension.multitenancy
  2. Configuration API: Spring-based → Programmatic MultiTenancyConfigurer
  3. TenantDescriptor: Simplified to data holder (removed configuration methods)
  4. Segment factories: New factory pattern for per-tenant infrastructure

Files Changed

  • Core module (multitenancy/) - Framework-agnostic multi-tenancy support
  • Spring module (multitenancy-spring/) - Spring Data JPA integration
  • Spring Boot module (multitenancy-spring-boot-autoconfigure/) - Auto-configuration
  • Axon Server connector (multitenancy-axon-server-connector/) - Distributed support
  • Integration tests (4 modules) - Comprehensive test coverage
  • Documentation (docs/) - 15 AsciiDoc files

Checklist

  • Code compiles and all tests pass
  • Package structure aligned with AF5 conventions
  • Programmatic configuration works without Spring
  • Spring Boot auto-configuration works
  • Axon Server integration tested (with Docker)
  • TenantComponent injection working
  • CommandDispatcher tenant propagation verified
  • Documentation covers major use cases
  • JPA event store testing (not personally verified)

theoema added 28 commits January 1, 2026 06:01
This commit migrates the foundation layer and CommandBus to AF5:

Core components migrated:
- TenantDescriptor, TenantProvider, MultiTenantAwareComponent
- TargetTenantResolver (Message no longer generic in AF5)
- NoSuchTenantException, TenantConnectPredicate
- MultiTenantCommandBus (fully rewritten for AF5 API)
- TenantCommandSegmentFactory

Key AF5 API changes applied:
- CommandBus.dispatch() now returns CompletableFuture<CommandResultMessage>
- dispatch() takes ProcessingContext parameter (nullable)
- subscribe() uses QualifiedName instead of String
- Message interface no longer takes type parameters
- javax.annotation -> jakarta.annotation
- org.axonframework.commandhandling -> org.axonframework.messaging.commandhandling

Build configuration updated:
- Java 21 (was Java 8)
- Axon Framework 5.1.0-SNAPSHOT
- Spring Framework 6.2.15
- Spring Boot 3.5.9

Pending migration (moved to pending_migration/):
- EventStore, QueryBus, EventProcessor components
- DeadLetterQueue, EventScheduler components
- Configuration module
- Interceptor support (to be re-added with new AF5 patterns)
Key changes:
- query() now returns MessageStream<QueryResponseMessage> with ProcessingContext
- subscribe() uses QualifiedName + QueryHandler, returns this (fluent)
- Removed scatterGather() and streamingQuery() - replaced by MessageStream
- Added subscription query methods: subscriptionQuery(), subscribeToUpdates()
- Added update emission methods (previously on QueryUpdateEmitter):
  emitUpdate(), completeSubscriptions(), completeSubscriptionsExceptionally()
- QueryMessage no longer takes type parameters
- Implemented DescribableComponent.describeTo()
AF5 EventStore is fundamentally different - no longer aggregate-centric.

Key changes:
- publish() now takes ProcessingContext, returns CompletableFuture<Void>
- subscribe() takes BiFunction returning CompletableFuture
- open() replaces openStream() - uses StreamingCondition instead of TrackingToken
- Added transaction(ProcessingContext) for EventStoreTransaction access
- Removed aggregate-centric methods (readEvents, storeSnapshot) - not in AF5
- Token operations delegate to tenant segments or throw if not resolvable

Added axon-eventsourcing dependency to pom.xml.
Key changes:
- Package: org.axonframework.messaging.eventhandling.processing.EventProcessor
- name() instead of getName()
- start() and shutdown() now return CompletableFuture<Void>
- Removed lifecycle annotations (@StartHandler, @ShutdownHandler)
- Added DescribableComponent.describeTo()
- Simplified by removing interceptor support (can be added later)
Simple predicate interface to determine if an event processor should
be multi-tenant. Portable as-is from AF4.

Note: MultiTenantEventProcessingModule requires complete redesign for
AF5's new configuration architecture and is deferred to Spring Boot
autoconfigure module.
Documents:
- Completed work (15 source files migrated)
- Remaining work with priorities
- Deferred changes with reasons
- Critical AF5 API changes reference
- Package/import changes
- Wisdom for tricky migrations
- Build commands and file references
Move multi-tenant components from components/ to messaging/ and
eventsourcing/ packages following AF5's package organization:

- messaging/commandhandling: MultiTenantCommandBus
- messaging/queryhandling: MultiTenantQueryBus
- messaging/eventhandling/processing: MultiTenantEventProcessor
- eventsourcing/eventstore: MultiTenantEventStore

Each component now implements the AF5 interfaces directly and uses
the new MessageStream-based API for async operations.

Add DECORATION_ORDER constants to each component to ensure proper
ordering in the AF5 decorator chain where InterceptingXxxBus wraps
MultiTenantXxxBus.
Introduce MultiTenancyConfigurer and MultiTenancyConfigurationDefaults
for integrating multi-tenancy with AF5's configuration API.

MultiTenancyConfigurer provides a fluent API for registering:
- TenantProvider for tenant lifecycle management
- TargetTenantResolver for message-based tenant resolution
- Segment factories for CommandBus, QueryBus, EventStore, EventProcessor

MultiTenancyConfigurationDefaults registers decorators that replace
standard infrastructure components with their multi-tenant equivalents
when the required components are configured.

The decoration order (MIN_VALUE + 50) ensures multi-tenant decorators
run before InterceptingXxxBus decorators (MIN_VALUE + 100), resulting
in the chain: Intercepting -> MultiTenant -> TenantSegments
Add tests demonstrating that InterceptingCommandBus and InterceptingQueryBus
correctly wrap MultiTenantCommandBus and MultiTenantQueryBus respectively.

Tests verify:
- Dispatch interceptors are invoked before tenant routing
- Interceptors can modify messages before they reach tenant segments
- Multiple interceptors chain in the correct order
- Interceptors can short-circuit dispatch
- Results flow back through the interceptor chain
- Subscription queries work with interception (QueryBus)

Test structure follows framework conventions with interception/
subpackages mirroring the core framework's test organization.
Remove test files for components that have not yet been migrated:
- TenantWrappedTransactionManager
- MultiTenantDeadLetterProcessor
- MultiTenantDeadLetterQueue
- MultiTenantQueryUpdateEmitter
- MultiTenantEventScheduler

These components require AF5 equivalents that are not yet available
or need further investigation before migration can proceed.
This commit introduces the foundational components for multi-tenant support
in Axon Framework 5. These components enable applications to isolate data
and operations by tenant while sharing the same application infrastructure.

Core Components:
- TenantDescriptor: Immutable value object identifying a tenant with an ID
  and optional properties. Uses Java record for cleaner API.
- TenantProvider: Contract for components that provision tenants and notify
  MultiTenantAwareComponents of tenant lifecycle changes.
- SimpleTenantProvider: Basic implementation for static tenant configurations
  where tenants are known at startup.
- MultiTenantAwareComponent: Interface for components that manage per-tenant
  segments (command buses, event stores, etc.).

Tenant Resolution:
- TargetTenantResolver: Strategy interface for resolving which tenant a
  message belongs to, enabling flexible tenant identification.
- MetadataBasedTenantResolver: Default implementation that extracts tenant
  from message metadata using a configurable key (default: "tenantId").
- TenantConnectPredicate: Predicate for filtering which tenants should be
  connected to multi-tenant components.

Exception Handling:
- NoSuchTenantException: Thrown when operations target an unregistered tenant.
- NoTenantInMessageException: Thrown when tenant cannot be resolved from a
  message, helping developers identify missing tenant metadata.

These components form the foundation that other multi-tenant infrastructure
(command bus, query bus, event store, event processors) builds upon.
This commit introduces a powerful system for injecting tenant-scoped
dependencies directly into message handlers. This eliminates boilerplate
code for looking up tenant-specific resources like repositories, services,
or entity managers.

Core Components:
- TenantComponentFactory: Functional interface for creating component
  instances per tenant. Enables lazy, on-demand component creation.
- TenantComponentRegistry: Manages tenant-scoped component instances,
  implementing MultiTenantAwareComponent for lifecycle management.
  Automatically creates/destroys component instances as tenants are
  registered/unregistered.

Parameter Resolution:
- TenantComponentResolver: Resolves tenant-scoped components as handler
  method parameters based on the current message's tenant context.
- TenantComponentResolverFactory: ParameterResolverFactory that creates
  resolvers for registered tenant component types.

Processing Context:
- TenantAwareProcessingContext: Wrapper that provides tenant context
  to the processing pipeline, enabling tenant resolution during
  message handling.
- TenantAwareProcessingContextResolver/Factory: Enables injecting the
  TenantAwareProcessingContext directly into handlers for advanced
  tenant-aware operations.

Example usage in a handler:
  @eventhandler
  void on(OrderCreated event, OrderRepository repository) {
      // repository is automatically tenant-scoped based on event metadata
      repository.save(new OrderProjection(event));
  }

This system integrates with the MultiTenantPooledStreamingEventProcessorModule
to provide seamless tenant-scoped dependency injection.
This commit provides multi-tenant routing for commands and queries,
enabling applications to dispatch messages to tenant-specific bus
segments transparently.

Command Bus:
- MultiTenantCommandBus: Routes commands to tenant-specific CommandBus
  segments based on message metadata. Implements the decorator pattern
  to wrap the standard command bus infrastructure.
- TenantCommandSegmentFactory: Factory interface for creating per-tenant
  CommandBus instances.
- TenantAwareCommandBus: Interface marker for command buses that are
  tenant-aware, providing access to the current tenant context.

Query Bus:
- MultiTenantQueryBus: Routes queries to tenant-specific QueryBus segments.
  Supports all query types including point-to-point, scatter-gather, and
  streaming queries.
- TenantQuerySegmentFactory: Factory interface for creating per-tenant
  QueryBus instances.

Key Features:
- Transparent tenant routing based on message metadata
- Dynamic tenant registration/unregistration via MultiTenantAwareComponent
- Automatic propagation of handler subscriptions to new tenant segments
- Full compatibility with Axon Framework 5's new messaging APIs

The multi-tenant buses resolve the target tenant from each message using
the configured TargetTenantResolver, then delegate to the appropriate
tenant segment. This enables complete data isolation between tenants
while sharing application infrastructure.
This commit provides multi-tenant event storage, ensuring complete
isolation of event streams between tenants while enabling event-sourced
aggregates to work transparently in a multi-tenant environment.

Core Components:
- MultiTenantEventStore: Routes event operations to tenant-specific
  EventStore segments. Resolves tenant from command messages during
  append operations, ensuring events are stored in the correct tenant's
  event store.
- TenantEventSegmentFactory: Factory interface for creating per-tenant
  EventStore instances.
- TenantEventStoreProvider: Interface for accessing tenant segments,
  used by event processors to obtain tenant-specific event sources.

JPA Implementation:
- JpaTenantEventSegmentFactory: Creates JPA-backed EventStore instances
  per tenant using tenant-specific EntityManagerFactory instances.
  Supports configurable storage engine settings per tenant.

Key Features:
- Complete event stream isolation between tenants
- Automatic tenant resolution from command context during writes
- Support for both embedded (JPA) and distributed (Axon Server) storage
- Caching of tenant segments for performance
- Integration with framework's event store decorator chain

The MultiTenantEventStore implements DECORATION_ORDER to ensure it runs
after correlation data interceptors apply tenant metadata, enabling
proper tenant resolution during event storage.
This commit provides multi-tenant event processing, enabling applications
to run separate event processor instances per tenant with isolated tracking
tokens and event streams.

Core Components:
- MultiTenantEventProcessor: Wrapper that manages per-tenant EventProcessor
  instances, delegating start/stop/status operations to all tenant segments.
- TenantEventProcessorSegmentFactory: Factory for creating per-tenant
  EventProcessor instances.
- MultiTenantEventProcessorPredicate: Predicate for determining which
  processors should be multi-tenant aware.

Pooled Streaming Module:
- MultiTenantPooledStreamingEventProcessorModule: Configuration module
  following Axon 5's module pattern. Creates multi-tenant pooled streaming
  processors with per-tenant event sources and token stores.
- MultiTenantPooledStreamingEventProcessorConfiguration: Configuration
  extending PooledStreamingEventProcessorConfiguration with tenant-specific
  settings like token store factory.

Token Store Factories:
- TenantTokenStoreFactory: Interface for creating per-tenant TokenStore
  instances, ensuring tracking tokens are isolated per tenant.
- InMemoryTenantTokenStoreFactory: In-memory implementation for testing
  and simple deployments.
- JdbcTenantTokenStoreFactory: JDBC-based implementation using tenant-
  specific database connections.
- JpaTenantTokenStoreFactory: JPA-based implementation using tenant-
  specific EntityManagerFactory instances.
- TenantConnectionProviderFactory: Factory for tenant-specific JDBC
  ConnectionProviders.

Key Features:
- Per-tenant event streaming with isolated tracking tokens
- Dynamic tenant processor creation/destruction
- Support for tenant-scoped component injection in handlers
- Integration with framework's processor configuration system
- Layered configuration (shared -> type-specific -> instance-specific)

Example usage:
  configurer.eventProcessing(ep -> ep
      .pooledStreaming(ps -> ps.processor(
          MultiTenantPooledStreamingEventProcessorModule.create("orders")
              .eventHandlingComponents(c -> c.autodetected(cfg -> new OrderProjector()))
              .tenantComponent(OrderRepository.class, tenant -> new OrderRepo(tenant))
      ))
  );
This commit provides the configuration API for enabling multi-tenancy
and the auto-discovery mechanism for registering default components.

Configuration API:
- MultiTenancyConfigurer: Fluent API for enhancing an Axon configuration
  with multi-tenancy support. Provides methods for:
  - Registering custom TenantProvider implementations
  - Configuring tenant resolvers for different message types
  - Registering tenant-scoped components
  - Enabling multi-tenant command bus, query bus, and event store

Auto-Discovery:
- MultiTenancyConfigurationDefaults: ConfigurationEnhancer discovered via
  SPI that automatically registers default multi-tenancy components when
  the extension is on the classpath. Registers:
  - MetadataBasedTenantResolver as the default tenant resolver
  - MultiTenantCommandBus decorator
  - MultiTenantQueryBus decorator
  - MultiTenantEventStore decorator
  - InMemoryTenantTokenStoreFactory as default (for embedded mode)

The defaults use registerIfNotPresent() semantics, allowing users to
override any component with custom implementations. The Axon Server
connector module provides its own enhancer with higher priority to
register distributed-mode defaults when Axon Server is detected.

Example usage:
  Configurer configurer = Configurer.withDefaults();

  MultiTenancyConfigurer.enhance(configurer)
      .tenantProvider(myTenantProvider)
      .tenantComponent(MyService.class, tenant -> new MyService(tenant));
This commit provides Axon Server-specific implementations for multi-tenant
deployments, enabling automatic tenant discovery from Axon Server contexts
and distributed command/query routing across tenant-specific connections.

Tenant Provider:
- AxonServerTenantProvider: Discovers tenants from Axon Server contexts
  using the admin API. Monitors context changes in real-time and notifies
  MultiTenantAwareComponents when tenants are added or removed. Supports:
  - Pre-defined contexts via configuration
  - Dynamic context discovery
  - Context filtering via TenantConnectPredicate
  - Graceful handling of connection failures

Distributed Bus Connectors:
- MultiTenantAxonServerCommandBusConnector: Routes commands to tenant-
  specific AxonServerCommandBusConnector instances. Uses the wrapping
  pattern to ensure framework decorators (like PayloadConvertingConnector)
  are applied automatically.
- MultiTenantAxonServerQueryBusConnector: Routes queries to tenant-
  specific AxonServerQueryBusConnector instances, supporting all query
  types including scatter-gather and streaming.

Event Store:
- AxonServerTenantEventSegmentFactory: Creates per-tenant EventStore
  instances backed by AxonServerEventStorageEngine, connecting each
  tenant to its own Axon Server context.

Auto-Discovery:
- DistributedMultiTenancyConfigurationDefaults: ConfigurationEnhancer
  with higher priority than the embedded defaults. When Axon Server is
  detected, registers distributed implementations for TenantProvider,
  CommandBusConnector, QueryBusConnector, and TenantEventSegmentFactory.

The connector implements the wrapping pattern for distributed buses:
rather than creating per-tenant DistributedCommandBus instances, it
provides a single multi-tenant connector that routes to tenant-specific
Axon Server connections. This ensures all framework infrastructure
(serialization, conversion, metrics) is shared across tenants.
This commit provides utilities for creating tenant-scoped Spring Data JPA
repositories, enabling seamless database-per-tenant architectures with
automatic repository injection into event handlers.

DataSource Management:
- TenantDataSourceProvider: Functional interface for providing tenant-
  specific DataSource instances. Implementations can use any strategy
  (connection pools per tenant, schema switching, etc.).

Entity Manager:
- TenantEntityManagerFactoryBuilder: Fluent builder for creating per-tenant
  EntityManagerFactory instances. Supports configurable:
  - Packages to scan for JPA entities
  - JPA/Hibernate properties
  - Persistence unit naming
  EMF instances are cached per tenant for efficiency.

Transaction Management:
- TenantTransactionManagerBuilder: Creates per-tenant TransactionManager
  instances that wrap Spring's JpaTransactionManager. Ensures each tenant's
  operations use the correct database connection.

Repository Creation:
- TenantRepositoryFactory: Simple factory for creating tenant-scoped
  repositories using Spring Data JPA's JpaRepositoryFactory.
- TenantJpaRepositoryFactory: Advanced factory that wraps repositories
  with automatic transaction management, following Axon's pattern of
  component-managed transactions.

Parameter Resolution:
- TenantRepositoryParameterResolverFactory: Enables injecting tenant-
  scoped repositories directly as handler method parameters. The framework
  automatically resolves the correct tenant from message metadata.

Example usage:
  // Configure tenant-scoped repositories
  TenantEntityManagerFactoryBuilder emfBuilder = TenantEntityManagerFactoryBuilder
      .forDataSourceProvider(tenant -> getDataSource(tenant))
      .packagesToScan("com.example.projections")
      .build();

  // In event handlers, repositories are auto-injected
  @eventhandler
  void on(OrderCreated event, OrderRepository repository) {
      repository.save(new OrderProjection(event)); // Uses tenant's database
  }
Provides comprehensive Spring Boot integration for the Axon Framework 5
multitenancy extension with automatic configuration of:

Multi-tenant event processing:
- MultiTenantEventProcessingAutoConfiguration creates multi-tenant pooled
  streaming event processors via MultiTenantMessageHandlerLookup
- MultiTenantMessageHandlerConfigurer uses MultiTenantEventProcessorPredicate
  to determine per-processor multi-tenancy enablement
- Replaces standard InfrastructureConfiguration with multi-tenant aware version

Tenant component injection:
- TenantComponentAutoConfiguration discovers TenantComponent implementations
  via classpath scanning and registers them for handler parameter injection
- TenantComponent interface in spring module provides factory pattern for
  creating tenant-specific instances with Spring dependency injection

Spring Data JPA support:
- MultiTenancySpringDataJpaAutoConfiguration auto-registers repository
  interfaces as tenant components when tenant-repositories is enabled
- MultiTenancyAutoConfigurationImportFilter excludes conflicting JPA
  autoconfiguration when per-tenant datasources are used

Configuration:
- MultiTenancyProperties provides Spring Boot property binding for
  tenant key, Axon Server contexts, and JPA repository settings
- TenantCorrelationProvider propagates tenant context between messages
- Updated autoconfiguration ordering and conditional bean registration

All autoconfigurations integrate via SPI-based ConfigurationEnhancer pattern
to properly interact with Axon Framework 5's component registry.
Comprehensive integration test suite covering:

Embedded (in-memory) tests:
- multitenancy-integration-tests-embedded: Core multi-tenancy tests without
  external dependencies using SimpleTenantProvider and in-memory event store
- Verifies tenant isolation, dynamic tenant registration, and same-entity-id
  across tenants without conflicts

Axon Server tests:
- multitenancy-integration-tests-axon-server: Tests against real Axon Server
  using Testcontainers (limited to 'default' context without license)
- Verifies command → event → projection → query flow through distributed
  multi-tenant infrastructure

Spring Boot Embedded tests:
- multitenancy-integration-tests-springboot-embedded: Spring Boot integration
  with embedded event store and per-tenant JPA repositories
- Tests TenantComponent injection (TenantAuditService) demonstrating tenant-
  scoped dependency injection into message handlers

Spring Boot Axon Server tests:
- multitenancy-integration-tests-springboot-axonserver: Full Spring Boot stack
  with Axon Server and tenant-scoped JPA projections
- Verifies autoconfiguration correctly wires multi-tenant infrastructure

All tests use a consistent Course domain model demonstrating:
- Event-sourced aggregates with tenant isolation
- Query-side projections with per-tenant repositories
- TenantComponent factory pattern for tenant-scoped services
Core tests:
- TenantDescriptorTest: Tenant identifier value object
- SimpleTenantProviderTest: Dynamic tenant registration and lifecycle
- TenantComponentRegistryTest: Per-tenant component caching and cleanup
- MetadataBasedTenantResolverTest: Message metadata tenant extraction

Configuration tests:
- MultiTenancyConfigurerTest: Fluent configuration API
- MultiTenancyConfigurationDefaultsTest: SPI-based auto-configuration

Event store tests:
- JpaTenantEventSegmentFactoryTest: Per-tenant JPA event storage

Token store factory tests:
- InMemoryTenantTokenStoreFactoryTest: In-memory token stores
- JdbcTenantTokenStoreFactoryTest: JDBC-based token stores
- JpaTenantTokenStoreFactoryTest: JPA-based token stores

Event processor tests:
- MultiTenantPooledStreamingEventProcessorConfigurationTest
- MultiTenantPooledStreamingEventProcessorModuleTest

Updates existing tests to use new package structure and APIs.
SPI registration:
- META-INF/services/ConfigurationEnhancer for auto-discovery of
  MultiTenancyConfigurationDefaults and DistributedMultiTenancyConfigurationDefaults

Axon Server connector tests:
- AxonServerTenantEventSegmentFactoryTest: Per-tenant event segment creation
- MultiTenantAxonServerCommandBusConnectorTest: Distributed command routing
- MultiTenantAxonServerQueryBusConnectorTest: Distributed query routing

Tests verify tenant isolation in distributed Axon Server deployments.
Removes obsolete code from the v4 implementation that has been replaced
by the new Axon Framework 5 architecture:

Core module:
- components/ package (moved to core/ with new API)
- configuration/ package (replaced by core/configuration/)
- Old tests for removed components

Autoconfigure module:
- AxonServerTenantProvider (moved to axon-server-connector module)
- MultiTenantDataSourceManager (replaced by TenantDataSourceProvider)
- MultiTenantEventProcessorControlService (not yet migrated)
- MultiTenantPersistentStream* (not yet migrated)
- Old autoconfigure tests

Old integration tests:
- multitenancy-spring-boot-3-integrationtests (replaced by new test modules)

Migration artifacts:
- pending_migration/ staging directory
- MIGRATION_SPEC.md and AXON5_MIGRATION_GUIDE.md working documents
Root pom.xml:
- Add new integration test modules to reactor build
- Update module ordering for proper dependency resolution

coverage-report/pom.xml:
- Include new modules in coverage aggregation

multitenancy-spring-boot-starter/pom.xml:
- Simplify starter dependencies for AF5 architecture
- Remove obsolete transitive dependency management
Align package structure with other Axon Framework extensions by using
singular 'extension' instead of plural 'extensions' in package names.

Changes:
- org.axonframework.extensions.multitenancy -> org.axonframework.extension.multitenancy
- Updated all imports and package declarations across all modules
- Updated META-INF service files to reflect new package paths

This brings the multitenancy extension in line with the spring extension
which uses org.axonframework.extension.springboot.
Complete rewrite of multitenancy extension documentation for AF5:

Setup Guides:
- Getting Started: Standard AF5 with in-memory event store
- Axon Server Integration: AF5 with Axon Server
- Spring Boot Integration: Spring Boot without Axon Server
- Spring Boot with Axon Server: The recommended production setup

Concepts:
- Tenant Management: TenantProvider and dynamic tenants
- Message Routing: TargetTenantResolver and metadata-based routing
- Infrastructure: Multi-tenant buses and event stores
- Projections: Tenant-scoped JPA and repository injection

Configuration Reference:
- Tenant Resolution: Custom resolvers and correlation
- Event Processors: Multi-tenant streaming processors
- Tenant Components: Per-tenant dependency injection
- Dynamic Tenants: Runtime tenant provisioning

Key patterns demonstrated:
- Separate state entities from command handlers (AF5 style)
- Method parameter injection for tenant-scoped dependencies
- Async REST endpoints with CompletableFuture
- Proper CommandGateway.send(command, metadata) usage

Removes obsolete AF4 documentation files.
Update remaining references from extensions.multitenancy to
extension.multitenancy in SPI service files and Spring Boot
autoconfiguration.
Verify that when using CommandDispatcher from within a handler,
tenant context is properly propagated through the correlation data
mechanism to dispatched commands.

The test uses a stateful automation pattern:
- Event handler reacts to CourseCreated and dispatches a command
- Command handler resolves tenant-scoped NotificationService via
  ProcessingContext.component()
- Verifies each tenant's service only receives its own notifications
@theoema theoema requested a review from a team as a code owner January 5, 2026 22:27
@theoema theoema removed the request for review from a team January 5, 2026 22:27
@CLAassistant
Copy link

CLAassistant commented Jan 5, 2026

CLA assistant check
All committers have signed the CLA.

This commit adds autoconfiguration support for tenant-scoped JDBC and
R2DBC database access, providing lightweight alternatives to JPA for
projections and queries.

JDBC support (axon.multi-tenancy.jdbc.enabled=true):
- Registers JdbcTemplate as tenant-scoped component
- Registers NamedParameterJdbcTemplate as tenant-scoped component
- Uses existing TenantDataSourceProvider for DataSource resolution

R2DBC support (axon.multi-tenancy.r2dbc.enabled=true):
- Adds TenantConnectionFactoryProvider interface for tenant-specific
  R2DBC ConnectionFactory instances
- Registers DatabaseClient as tenant-scoped component
- Enables non-blocking database operations in message handlers

Both can be injected directly into @eventhandler methods and are
automatically scoped to the tenant from message metadata.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants