Skip to content

Configuration Options

Dmytro Vyazelenko edited this page Sep 23, 2024 · 129 revisions

Aeron provides a number of configuration options to control its behaviour at runtime. There are 3 major ways by which configuration options can be applied with the following order of precedence.

  1. Channel URI Params: Parameters applied to the end of the URI string when adding Publications or Subscriptions.
  2. Context Object Properties: Properties on the context object when creating the Aeron Client or MediaDriver programmatically.
  3. System Properties: Properties defined on the command line or loaded from properties files. For length/size values it is possible to use the full number such as 16384 or a shorten form such as 16k with a suffix of g, m, or k respectively meaning gigabytes, megabytes, and kilobytes. For duration values in nanoseconds the short form can use the suffix s, ms, us, or ns for seconds, milliseconds, microseconds, and nanoseconds respectively.

A full listing of configurations options can be found by following the link.

Channel URI Params

When adding Publications or Subscriptions via the client, URI params can be appended to the channel string which will override system wide configuration options. Over time more options will be available via this mechanism.

  • ttl: Sets the Time To Live for multicast datagrams, e.g. aeron:udp?endpoint=224.0.1.1:40456|ttl=16
  • mtu: Sets the Maximum Transmission Unit for the length of a multicast datagram, e.g. aeron:udp?endpoint=224.0.1.1:40456|mtu=4096

For complete listing of the channel URI params please go here.

Context Object Properties

The Aeron client and MediaDriver can be programmatically launched with a context object provided for configuration. If properties are set on the context object before it is concluded then they will take precedence over system properties. The Aeron Client and MediaDriver have their own context objects. Properties can be set via a fluent API and then passed in when launching the client or media driver.

Configuring a client example.

    final Aeron.Context ctx = new Aeron.Context();
    if (EMBEDDED_MEDIA_DRIVER)
    {
        ctx.aeronDirectoryName(driver.aeronDirectoryName());
    }

    try (Aeron aeron = Aeron.connect(ctx);
         Publication publication = aeron.addPublication(CHANNEL, STREAM_ID))
    {
        // use publication
    }

Configuring a driver example.

/**
 * Sample setup for a {@link MediaDriver} that is configured for low latency communications. This configuration
 * requires sufficient CPU resource to delivery low latency performance, i.e. 3 active polling threads.
 */
public class LowLatencyMediaDriver
{
    public static void main(final String[] args)
    {
        loadPropertiesFiles(args);

        final MediaDriver.Context ctx = new MediaDriver.Context()
            .termBufferSparseFile(false)
            .useWindowsHighResTimer(true)
            .threadingMode(ThreadingMode.DEDICATED)
            .conductorIdleStrategy(BusySpinIdleStrategy.INSTANCE)
            .receiverIdleStrategy(NoOpIdleStrategy.INSTANCE)
            .senderIdleStrategy(NoOpIdleStrategy.INSTANCE);

        try (MediaDriver ignored = MediaDriver.launch(ctx))
        {
            new ShutdownSignalBarrier().await();

            System.out.println("Shutdown Driver...");
        }
    }
}

System Properties

The most common way of configuring Aeron is via systems properties. System properties can be set on the Java command line by prefixing the property with -D followed by the property name equals property value as in the following example

$ java -cp aeron-all/build/libs/aeron-all-<version>.jar \
    -Daeron.mtu.length=8k \
    io.aeron.driver.MediaDriver

It is often more convenient packaging up a set of properties into a properties file. Aeron can load a list of properties files passed in as command line arguments to the MediaDriver. Each will be loaded in turn adding to the set of system properties. Some sample properties files are provided as resources in the driver module.

The filenames will be treated as a URI and the driver will attempt to load the properties as resources, local files, and remotely via HTTP in that order. The following is an example of using multiple configuration properties.

$ java -cp aeron-all/build/libs/aeron-all-<version>.jar \
    -Daeron.mtu.length=8k \
    io.aeron.driver.MediaDriver \
    aeron-throughput.properties \
    http://config.server.local/aeron-deployment.properties \
    ~/conf/aeron/my-test.properties

C Media Driver

Aeron has two implementations of the Media Driver, one implemented in Java the other implemented in C. While we endeavour to ensure parity between the two versions, some of the functionality differs. Therefore some of the configuration options, either only exist for one of the drivers or do exist, but work differently. E.g. using a supplier function pointer in the C driver instead of a supplier interface in the Java one. We will highlight any options that are different or not available as appropriate.

The C driver can be configured three ways. The Java format -D<option>=<value> is supported in the C media driver as command line options. Environment variables can also be used. To convert from the system property name to the environment variables, upper case all of the alpha characters and replace . with _ (this is what the C media driver does internally) treats the Java configuration options (E.g. aeron.print.configuration would become AERON_PRINT_CONFIGURATION).

The C media driver can also be embedded. In this case the configuration options supported are environment variables and setting of the values manually using the aeronmd.h API, in a similar fashion to embedding the Java driver. In this case the manual settings take precedence over the environment variables. When embedding the C function for setting you should be able to translate from the Java method to the C function name. Prefix the method name with set or get as appropriate, convert from camel case to snake case, and prefix with aeron_driver_context. E.g. MediaDriver.Context.threadingMode would become aeron_driver_context_set_threading_mode.

Options List

  1. Common Options
  2. Aeron Client Options
  3. Media Driver Options
  4. Channel Endpoint Extension Options
  5. Aeron Archive Client Options
  6. Aeron Cluster Client Options

Common Options

Common options are those used both by the Aeron client and the Media Driver.

Aeron Directory
Description Directory under which the driver will store all its files.
Type String
Default The users TMP/TEMP directory if not Linux otherwise /dev/shm, i.e. resulting in <java.io.tmp>/aeron-<user.name> when driver launched standalone, and <java.io.tmp>/aeron-<user.name>-<randomUUID>.
System Property aeron.dir
Context CommonContext.aeronDirectoryName()
URI Param N/A
Driver Timeout
Description The timeout in milliseconds after which the driver is considered dead if it does not update its C'n'C timestamp.
Type int64
Default 10000 (10 seconds)
System Property aeron.driver.timeout
Context CommonContext.driverTimeoutMs()
URI Param N/A
Enable Experimental Features
Description Determine if new experimental features should be enabled.
Type boolean (bool)
Default false
System Property aeron.enable.experimental.features
Context CommonContext.enableExperimentalFeatures()
URI Param N/A
Since Version 1.44.0

Aeron Client Options

Within the client process there are a small number of available options. The goal is to keep this list to a minimum with instruction taken from the Media Driver.

Client Name
Description The name of the client.
Type String
Default Empty
System Property aeron.client.name
Context io.aeron.Aeron.Context#clientName()
URI Param N/A
Since Version 1.44.0
Client Error Handler
Description The error handler to be called by the client conductor when it detects and issue with the driver. It is recommended that users set their own error handler.
Type org.agrona.ErrorHandler
Default Prints a stack trace to STDERR and then exits the process if exception is an instance of DriverTimeoutException.
System Property N/A
Context Aeron.Context.errorHandler()
URI Param N/A
Client Interservice Timeout
Description The timeout in nanoseconds that if exceeded by client conductor duty cycles the client will consider itself a zombie and self suicide by closing publications and images then invoking the error handler.
Type int64
Default 10_000_000_000 (10 seconds) and set as default from the MediaDriver.Context.clientLivenessTimeoutNs() (i.e. aeron.client.liveness.timeout) via the CnC file.
System Property N/A
Context Aeron.Context.interServiceTimeout()
Client Keepalive Interval
Description The interval time in nanoseconds for which the client will send a keepalive messages on the control protocol to the driver to indicate it is still here.
Type int64
Default 500_000_000 (500ms)
System Property N/A
Context Aeron.Context.keepAliveInterval()
URI Param N/A
Resource Linger Duration
Description Duration in nanoseconds to wait while lingering a entity such as an Image before deleting underlying resources such as memory mapped files.
Type int64
Default 3_000_000_000 (3 seconds)
System Property aeron.client.resource.linger.duration
Context Aeron.Context.resourceLingerDurationNs()
URI Param N/A
Close Linger Duration
Description Duration in nanoseconds to linger on closing to allow publishers and subscribers time to notice closed resources.
Type int64
Default 0
System Property aeron.client.close.linger.duration
Context Aeron.Context.closeLingerDurationNs()
URI Param N/A
Client Conductor Idle Strategy
Description Idle strategy to be used by the Client Conductor.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.SleepingMillisIdleStrategy (16 milliseconds)
System Property N/A
Context Aeron.Context.idleStrategy()
URI Param N/A
Awaiting Idle Strategy
Description Idle strategy to be used when awaiting a response from the Media Driver.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.SleepingMillisIdleStrategy (1 millisecond)
System Property N/A
Context Aeron.Context.awaitingIdleStrategy()
URI Param N/A
Client Lock
Description The Lock that is used to provide mutual exclusion in the Aeron client.
Type java.util.concurrent.locks.Lock
Default java.util.concurrent.locks.ReentrantLock
System Property N/A
Context Aeron.Context.clientLock()
URI Param N/A
Epoch Clock
Description The EpochClock to be used for tracking wall clock time when interacting with the driver.
Type org.agrona.concurrent.EpochClock
Default org.agrona.concurrent.SystemEpochClock
System Property N/A
Context Aeron.Context.epochClock()
URI Param N/A
Nano Clock
Description The NanoClock to be used for tracking high resolution time.
Type org.agrona.concurrent.NanoClock
Default org.agrona.concurrent.SystemNanoClock
System Property N/A
Context Aeron.Context.nanoClock()
URI Param N/A
Available Image Handler
Description Callback handler to be notified when an Image becomes available.
Type io.aeron.AvailableImageHandler
Default null
System Property N/A
Context Aeron.Context.availableImageHandler()
URI Param N/A
Unavailable Image Handler
Description Callback handler to be notified when an Image becomes unavailable.
Type io.aeron.UnavailableImageHandler
Default null
System Property N/A
Context Aeron.Context.unavailableImageHandler()
URI Param N/A
Available Counter Handler
Description Callback handler to be notified when a counter is available
Type io.aeron.AvailableCounterHandler
Default null
System Property N/A
Context Aeron.Context.availableCounterHandler()
URI Param N/A
Unavailable Counter Handler
Description Callback handler to be notified when a counter is unavailable.
Type io.aeron.UnavailableCounterHandler
Default null
System Property N/A
Context Aeron.Context.unavailableCounterHandler()
URI Param N/A
Use Conductor Agent Invoker
Description Should an AgentInvoker should be used for running the ClientConductor rather than run it on a thread with an AgentRunner.
Type boolean
Default false
System Property N/A
Context Aeron.Context.useConductorAgentInvoker()
URI Param N/A
Driver Agent Invoker
Description AgentInvoker for the Media Driver to be used while awaiting a synchronous response.
Type org.agrona.concurrent.AgentInvoker
Default null
System Property N/A
Context Aeron.Context.driverAgentInvoker()
URI Param N/A
Pre-Touch Mapped Memory
Description Should mapped-memory be pre-touched to avoid soft page faults.
Type boolean
Default false
System Property aeron.pre.touch.mapped.memory
Context Aeron.Context.preTouchMappedMemory()
URI Param N/A
Close Handler
Description Callback handler to be called when the client is closed by timeout or normal means.
Type java.lang.Runnable
Default null
System Property N/A
Context Aeron.Context.closeHandler()
URI Param N/A
Thread Factory
Description The thread factory to use when starting the conductor thread.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context Aeron.Context.threadFactory()
URI Param N/A

Media Driver Options

The bulk of configuration is for the Media Driver. Most options have defaults which the end user should not need to change. Socket buffers and flow control windows are the exception when throughput performance is directly influenced by Bandwidth Delay Product (BDP) which needs to be considered. Idle and threading strategies are useful to trading CPU usage for throughput and latency.

Print Configuration on Start Up
Description Should the driver print it's configuration on start to System#out at the end of conclude().
Type boolean
Default false
System Property aeron.print.configuration
Context MediaDriver.Context.printConfigurationOnStart()
C Driver aeron_driver_context_set_print_configuration()
URI Param N/A
Delete Aeron Directory on Start Up
Description Should the media driver delete the Aeron directory on startup even if the timestamp is current. This is useful for testing of forced restart.
Type boolean
Default false
System Property aeron.dir.delete.on.start
Context MediaDriver.Context.dirDeleteOnStart()
URI Param N/A
Delete Aeron Directory on Shutdown
Description Should the media driver delete the Aeron directory on shutdown.
Type boolean
Default false
System Property aeron.dir.delete.on.shutdown
Context MediaDriver.Context.dirDeleteOnShutdown()
URI Param N/A
Use High Resolution Timer on Windows (Java only)
Description Attempt to use the high resolution timer on Windows so parking is reduced from 15ms to 1ms.
Type boolean
Default false
System Property aeron.use.windows.high.res.timer
Context MediaDriver.Context.useWindowsHighResTimer()
URI Param N/A
Warn if Directory Exists
Description Should a warning be issued if the CommonContext.aeronDirectoryName() exists?
Type boolean
Default false
System Property aeron.dir.warn.if.exists
Context MediaDriver.Context.warnIfDirectoryExists()
C Driver aeron_driver_context_set_dir_warn_if_exists()
URI Param N/A
Media Driver Error Handler (Java only)
Description The error handler to be called when an exception occurs in the driver. Only override this with a specific usecase in mind.
Type org.agrona.ErrorHandler
Default Records to the DistinctErrorLog.
System Property N/A
Context MediaDriver.Context.errorHandler()
URI Param N/A
Media Driver Error Log (Java only)
Description Log to which exceptions are recorded.
Type org.agrona.concurrent.errors.DistinctErrorLog
Default org.agrona.concurrent.errors.DistinctErrorLog
System Property N/A
Context MediaDriver.Context.errorLog()
URI Param N/A
File Page Size
Description File page size for storage which can be adjusted to support huge pages. Files will be aligned and rounded up to this size.
Type int32
Default 4 * 1024
System Property aeron.file.page.size
Context MediaDriver.Context.filePageSize()
URI Param N/A
Sender wildcard port range
Description A wildcard port range for the receiver in the format low high. Used for replacing port 0 on a control address with a port from the configured range.
Type String
Default Empty
System Property aeron.sender.wildcard.port.range
Context MediaDriver.Context#senderWildcardPortRange()
URI Param N/A
Since Version 1.42.0
Receiver wildcard port range
Description A wildcard port range for the receiver in the format low high. Used for replacing port 0 on a subscription with a port from the configured range.
Type String
Default Empty
System Property aeron.receiver.wildcard.port.range
Context MediaDriver.Context#receiverWildcardPortRange()
URI Param N/A
Since Version 1.42.0
Term Buffer Length
Description The length in bytes of a publication buffer to hold a term of messages. It must be a power of 2 and be in the range of 64KB to 1GB.
Type (Java / C) int32 / size_t
Default 16 * 1024 * 1024
System Property aeron.term.buffer.length
Context MediaDriver.Context.publicationTermBufferLength()
URI Param term-length
IPC Term Buffer Length
Description The length in bytes of a IPC publication buffer to hold a term of messages. It must be a power of 2 and be in the range of 64KB to 1GB.
Type (Java / C) int32 / size_t
Default 64 * 1024 * 1024
System Property aeron.ipc.term.buffer.length
Context MediaDriver.Context.ipcTermBufferLength()
URI Param term-length
Publication Term Window Length
Description Default length in bytes for a term buffer window on a network publication.
Type (Java / C) int32 / size_t
Default 0
System Property aeron.publication.term.window.length
Context MediaDriver.Context.publicationTermWindowLength()
URI Param N/A
IPC Term Window Length
Description Default length in bytes for a term buffer window on a network publication.
Type (Java / C) int32 / size_t
Default 0
System Property aeron.ipc.publication.term.window.length
Context MediaDriver.Context.ipcPublicationTermWindowLength()
URI Param N/A
Conductor Idle Strategy (Java Driver)
Description Idle strategy to be used by the Driver Conductor agent in the media driver when in ThreadingMode.DEDICATED or ThreadingMode.SHARED_NETWORK.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.BackoffIdleStrategy
System Property aeron.conductor.idle.strategy
Context MediaDriver.Context.conductorIdleStrategy()
URI Param N/A
Conductor Idle Strategy (C Driver)
Description Idle strategy to be used by the Driver Conductor agent in the media driver when in AERON_THREADING_MODE_DEDICATED or AERON_THREADING_MODE_SHARED_NETWORK. Should be a string describing the strategy. E.g. sleeping, yielding, spinning, noop, backoff or a symbol name that resolves to a aeron_idle_strategy_t
Type const char *
Default "backoff"
System Property aeron.conductor.idle.strategy
Context aeron_driver_context_set_conductor_idle_strategy()
URI Param N/A
Conductor Idle Strategy Init Args (C Driver)
Description Initialisation parameters for the conductor idle strategies. E.g. a value 1ms with the sleeping strategy will configure it do sleep for 1ms on idle.
Type const char *
Default NULL
System Property aeron.conductor.idle.strategy
Context aeron_driver_context_set_conductor_idle_strategy_init_args()
URI Param N/A
Sender Idle Strategy (Java Driver)
Description Idle strategy to be used by the Sender agent in the media driver when in ThreadingMode.DEDICATED.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.BackoffIdleStrategy
System Property aeron.sender.idle.strategy
Context MediaDriver.Context.senderIdleStrategy()
URI Param N/A
Sender Idle Strategy (C Driver)
Description Idle strategy to be used by the Sender agent in the C media driver in AERON_THREADING_MODE_DEDICATED.
Type const char *
Default "backoff"
System Property aeron.sender.idle.strategy
Context aeron_driver_context_set_sender_idle_strategy()
URI Param N/A
Sender Idle Strategy Init Args (C Driver)
Description Initialisation parameters for the Sender idle strategies.
Type const char *
Default NULL
System Property aeron.conductor.sender.strategy
Context aeron_driver_context_set_sender_strategy_init_args()
URI Param N/A
Receiver Idle Strategy (Java Driver)
Description Idle strategy to be used by the Sender agent in the media driver when in ThreadingMode.DEDICATED.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.BackoffIdleStrategy
System Property aeron.receiver.idle.strategy
Context MediaDriver.Context.receiverIdleStrategy()
URI Param N/A
Receiver Idle Strategy (C Driver)
Description Idle strategy to be used by the Receiver agent in the C media driver in AERON_THREADING_MODE_DEDICATED.
Type const char *
Default "backoff"
System Property aeron.receiver.idle.strategy
Context aeron_driver_context_set_receiver_idle_strategy()
URI Param N/A
Receiver Idle Strategy Init Args (C Driver)
Description Initialisation parameters for the Receiver idle strategies.
Type const char *
Default NULL
System Property aeron.conductor.receiver.strategy
Context aeron_driver_context_set_receiver_strategy_init_args()
URI Param N/A
Shared Network Idle Strategy (Java Driver)
Description Idle strategy to be used by the Sender and Receiver agents in the media driver when in ThreadingMode.SHARED_NETWORK.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.BackoffIdleStrategy
System Property aeron.sharednetwork.idle.strategy
Context MediaDriver.Context.sharedNetworkIdleStrategy()
URI Param N/A
Shared Network Idle Strategy (C Driver)
Description Idle strategy to be used by the Shared Network agent in the C media driver in AERON_THREADING_MODE_SHARED_NETWORK.
Type const char *
Default "backoff"
System Property aeron.sharednetwork.idle.strategy
Context aeron_driver_context_set_sharednetwork_idle_strategy()
URI Param N/A
Shared Network Idle Strategy Init Args (C Driver)
Description Initialisation parameters for the Shared Network idle strategies.
Type const char *
Default NULL
System Property aeron.conductor.sharednetwork.strategy
Context aeron_driver_context_set_sharednetwork_strategy_init_args()
URI Param N/A
Shared Idle Strategy
Description Idle strategy to be used by the all agents in the media driver when in ThreadingMode.SHARED.
Type org.agrona.concurrent.IdleStrategy
Default org.agrona.concurrent.BackoffIdleStrategy
System Property aeron.shared.idle.strategy
Context MediaDriver.Context.sharedIdleStrategy()
URI Param N/A
Shared Idle Strategy (C Driver)
Description Idle strategy to be used by the Shared agent in the C media driver in AERON_THREADING_MODE_SHARED.
Type const char *
Default "backoff"
System Property aeron.shared.idle.strategy
Context aeron_driver_context_set_shared_idle_strategy()
URI Param N/A
Shared Idle Strategy Init Args (C Driver)
Description Initialisation parameters for the Shared idle strategies.
Type const char *
Default NULL
System Property aeron.conductor.shared.strategy
Context aeron_driver_context_set_shared_strategy_init_args()
URI Param N/A
Use Sparse Files For Term Buffers
Description Should term buffers be created as sparse files. If a platform supports sparse files then log buffer creation is faster with pages being allocated as needed. This can help for large numbers of channels/streams but can result in latency pauses.
Type Boolean
Default false
System Property aeron.term.buffer.sparse.file
Context MediaDriver.Context.termBufferSparseFile()
URI Param N/A
Perform Storage Checks
Description Should checks be performed on storage to ensure sufficient space for log buffers.
Type boolean
Default true
System Property aeron.perform.storage.checks
Context MediaDriver.Context.performStorageChecks()
URI Param N/A
Low File Store Warning Threshold (Java only)
Description Threshold in bytes below which driver will warn to STDERR that file space is running low.
Type int64
Default 10 times the aeron.term.buffer.length
System Property aeron.low.file.store.warning.threshold
Context MediaDriver.Context.lowStorageWarningThreshold()
URI Param N/A
Conductor Command Buffer Length
Description Length in bytes for the clients to driver conductor command buffer.
Type (Java / C) int32 / size_t
Default (1024 * 1024) + RingBufferDescriptor.TRAILER_LENGTH
System Property aeron.conductor.buffer.length
Context MediaDriver.Context.conductorBufferLength()
C Driver aeron_driver_context_to_conductor_buffer_length()
URI Param N/A
To Clients Broadcast Buffer Length
Description Length in bytes for broadcasting events from the driver to clients.
Type (Java / C) int32 / size_t
Default (1024 * 1024) + BroadcastBufferDescriptor.TRAILER_LENGTH
System Property aeron.clients.buffer.length
Context MediaDriver.Context.toClientsBufferLength()
C Driver aeron_driver_context_to_client_buffer_length()
URI Param N/A
Counters Buffer Length
Description Length in bytes for the buffer containing counters. May need to increase for large numbers of streams or subscriptions. Each counter requires 128 bytes to avoid false sharing.
Type (Java / C) int32 / size_t
Default 1024 * 1024
System Property aeron.counters.buffer.length
Context MediaDriver.Conext.counterValuesBufferLength()
C Driver
URI Param N/A
Error Buffer Length
Description Length in bytes for the buffer containing recorded errors. May need to increase if many different distinct exception traces are observed.
Type (Java / C) int32 / size_t
Default 1024 * 1024
System Property aeron.error.buffer.length
Context MediaDriver.Conext.errorBufferLength()
URI Param N/A
Loss Report Buffer Length
Description Length in bytes for the buffer containing recorded loss entries. Once full no more loss will be recorded. Disable by setting the value to zero.
Type (Java / C) int32 / size_t
Default 1024 * 1024
System Property aeron.loss.report.buffer.length
Context MediaDriver.Context.lossReportBufferLength()
URI Param N/A
Initial Receiver Window Length
Description Window for flow control of in-flight bytes between sender and receiver on a stream. This needs to be sufficient for BDP (Bandwidth Delay Product) and be greater than MTU.
Type (Java / C) int32 / size_t
Default 128 * 1024
System Property aeron.rcv.initial.window.length
Context MediaDriver.Context.initialWindowLength()
C Driver aeron_driver_context_set_rcv_initial_window_length()
URI Param N/A
Timer Interval
Description Interval in nanoseconds between checks for timers and timeouts.
Type (Java / C) int64 / uint64_t
Default 1_000_000_000 (1 second)
System Property aeron.timer.interval
Context MediaDriver.Context.timerIntervalNs()
URI Param N/A
Receiver Status Message Timeout
Description Timeout after which a status message will be sent if one has not been triggered by data flow.
Type (Java / C) int64 / uint64_t
Default 200_000_000 (200 milliseconds)
System Property aeron.rcv.status.message.timeout
Context MediaDriver.Context.statusMessageTimeoutNs()
C Driver aeron_driver_context_set_status_message_timeout_ns()
URI Param N/A
Receive Socket Buffer Length
Description Length in bytes for the SO_RCVBUF, 0 means use OS default. This needs to be larger than Receiver Window.
Type (Java / C) int32 / size_t
Default 128 * 1024
System Property aeron.socket.so_rcvbuf
Context MediaDriver.Context.socketRcvbufLength()
C Driver aeron_driver_context_set_socket_so_rcvbuf()
URI Param N/A
Sender Socket Buffer Length
Description Length in bytes for the SO_SNDBUF, 0 means use OS default. This needs to be larger than Receiver Window.
Type (Java / C) int32 / size_t
Default 0
System Property aeron.socket.so_sndbuf
Context MediaDriver.Context.socketSndbufLength()
C Driver aeron_driver_context_set_socket_so_sndbuf()
URI Param N/A
Multicast Time To Live (TTL)
Description Number of hops multicast will be allowed to propagate, zero means use OS default.
Type (Java / C) int32 / uint8_t
Default 0
System Property aeron.socket.multicast.ttl
Context MediaDriver.Context.socketMulticastTtl()
URI Param ttl
Publication Connection Timeout
Description Timeout in nanoseconds since the last status message after which a Publication will be considered not connected.
Type (Java / C) int64 / uint64_t
Default 5_000_000_000 (5 seconds)
System Property aeron.publication.connection.timeout
Context MediaDriver.Context.publicationConnectionTimeoutNs()
URI Param N/A
Publication Linger Timeout
Description Timeout in nanoseconds for a publication to linger after it is closed so it can respond to NAKs.
Type (Java / C) int64 / uint64_t
Default 5_000_000_000 (5 seconds)
System Property aeron.publication.linger.timeout
Context MediaDriver.Context.publicationLingerTimeoutNs()
URI Param N/A
Publication Unblock Timeout
Description Timeout in nanoseconds after which a publication will be unblocked if an offer is partially complete to allow other publishers to make progress.
Type (Java / C) int64 / uint64_t
Default 15_000_000_000 (15 seconds)
System Property aeron.publication.unblock.timeout
Context MediaDriver.Context.publicationUnblockTimeoutNs()
URI Param N/A
Untethered Window Limit Timeout
Description The timeout for when an untethered subscription that is outside the window will participate in local flow control.
Type (Java / C) int64 / uint64_t
Default 5_000_000_000 (5 seconds)
System Property aeron.untethered.window.limit.timeout
Context MediaDriver.Context.untetheredWindowLimitTimeoutNs()
URI Param N/A
Untethered Resting Timeout
Description Timeout for when an untethered subscription is resting after not being able to keep up before it is allowed to rejoin a stream.
Type (Java / C) int64 / uint64_t
Default 10_000_000_000 (10 seconds)
System Property aeron.untethered.resting.timeout
Context MediaDriver.Context.untetheredRestingTimeoutNs()
URI Param N/A
Retransmit Unicast Delay
Description Delay in nanoseconds before retransmitting after a NAK.
Type (Java / C) int64 / uint64_t
Default 0
System Property aeron.retransmit.unicast.delay
Context MediaDriver.Context.retransmitUnicastDelayNs()
URI Param N/A
Retransmit Unicast Delay Generator (Java Only)
Description FeedbackDelayGenerator for controlling the delay before sending a retransmit.
Type io.aeron.driver.FeedbackDelayGenerator
Default io.aeron.driver.StaticDelayGenerator using aeron.retransmit.unicast.delay
System Property N/A
Context MediaDriver.Context.retransmitUnicastDelayGenerator()
URI Param N/A
Retransmit Unicast Linger
Description Delay in nanoseconds for lingering after a retransmission.
Type (Java / C) int64 / uint64_t
Default 60_000_000 (60 ms)
System Property aeron.retransmit.unicast.linger
Context MediaDriver.Context.retransmitUnicastLingerNs()
URI Param N/A
Retransmit Unicast Linger Generator (Java Only)
Description FeedbackDelayGenerator for controlling the linger after a retransmit.
Type io.aeron.driver.FeedbackDelayGenerator
Default io.aeron.driver.StaticDelayGenerator using aeron.retransmit.unicast.linger
System Property N/A
Context MediaDriver.Context.retransmitUnicastLingerGenerator()
URI Param N/A
NAK Unicast Delay
Description Delay in nanoseconds before retransmission after an NAK on unicast.
Type (Java / C) int64 / uint64_t
Default 60_000_000 (60 ms)
System Property aeron.nak.unicast.delay
Context MediaDriver.Context.nakUnicastDelayNs()
URI Param N/A
Unicast Feedback Delay Generator (Java Only)
Description FeedbackDelayGenerator for controlling the delay of sending NAK feedback on unicast.
Type io.aeron.driver.FeedbackDelayGenerator
Default io.aeron.driver.StaticDelayGenerator using aeron.nak.unicast.delay
System Property N/A
Context MediaDriver.Context.unicastFeedbackDelayGenerator()
URI Param N/A
NAK Multicast Backoff
Description The maximum time in nanoseconds to backoff before sending a NAK on multicast.
Type (Java / C) int64 / uint64_t
Default 60_000_000 (60 ms)
System Property aeron.nak.multicast.max.backoff
Context MediaDriver.Context.nakMulticastMaxBackoffNs()
URI Param N/A
NAK Multicast Group Size
Description Estimate of the multicast receiver group size on a stream.
Type (Java / C) int32 / size_t
Default 10
System Property aeron.nak.multicast.group.size
Context MediaDriver.Context.nakMulticastGroupSize()
URI Param N/A
Multicast Feedback Delay Generator (Java Only)
Description FeedbackDelayGenerator for controlling the delay of sending NAK feedback on multicast.
Type io.aeron.driver.FeedbackDelayGenerator
Default io.aeron.driver.OptimalMulticastDelayGenerator using aeron.nak.multicast.max.backoff and aeron.nak.multicast.group.size
System Property N/A
Context MediaDriver.Context.multicastFeedbackDelayGenerator()
URI Param N/A
Client Liveness Timeout
Description Timeout in nanoseconds after which a driver will consider a client dead if it has not received an keepalive.
Type (Java / C) int64 / uint64_t
Default 10_000_000_000 (10 seconds)
System Property aeron.client.liveness.timeout
Context MediaDriver.Context.clientLivenessTimeoutNs()
URI Param N/A
Image Liveness Timeout
Description Timeout in nanoseconds for each the INACTIVE and LINGER stages an image will be retained for when it is no longer referenced.
Type (Java / C) int64 / uint64_t
Default 10_000_000_000 (10 seconds)
System Property aeron.image.liveness.timeout
Context MediaDriver.Context.imageLivenessTimeoutNs()
URI Param N/A
Counter Free to Reuse Timeout
Description Time in nanoseconds after which a freed counter may be reused.
Type (Java / C) int64 / uint64_t
Default 1_000_000_000 (1 second)
System Property aeron.counters.free.to.reuse.timeout
Context MediaDriver.Context.counterFreeToReuseTimeoutNs()
C Driver aeron_driver_context_counters_free_to_reuse_timeout_ns()
URI Param N/A
Unicast Flow Control Strategy (Java Only)
Description Strategy to be employed for flow control on UDP unicast streams which is loaded by the default supplier.
Type io.aeron.driver.FlowControl
Default io.aeron.driver.UnicastFlowControl
System Property aeron.unicast.flow.control.strategy
Context N/A
URI Param N/A
Unicast Flow Control Strategy Supplier (Java Driver)
Description Supplier of UDP unicast flow control strategies.
Type io.aeron.driver.FlowControlSupplier
Default io.aeron.driver.DefaultUnicastFlowControlSupplier
System Property aeron.unicast.FlowControl.supplier
Context MediaDriver.Context.unicastFlowControlSupplier()
URI Param N/A
Unicast Flow Control Strategy Supplier (C Driver)
Description Supplier of UDP unicast flow control strategies. Should be unicast_max or the name of a supplier function to dynamically load.
Type (property) const char *
Type (API) aeron_flow_control_strategy_supplier_func_t
Default "aeron_unicast_flow_control_strategy_supplier"
System Property aeron.unicast.FlowControl.supplier
Context aeron_driver_context_set_unicast_flowcontrol_supplier()
URI Param N/A
Multicast Flow Control Strategy (Java Only)
Description Strategy to be employed for flow control on UDP multicast streams which is loaded by the default supplier.
Type io.aeron.driver.FlowControl
Default io.aeron.driver.MaxMulticastFlowControl
System Property aeron.multicast.flow.control.strategy
Context N/A
URI Param N/A
Multicast Flow Control Strategy Supplier (Java Driver)
Description Supplier of UDP multicast flow control strategies.
Type io.aeron.driver.FlowControlSupplier
Default io.aeron.driver.DefaultMulticastFlowControlSupplier
System Property aeron.multicast.FlowControl.supplier
Context MediaDriver.Context.multicastFlowControlSupplier()
URI Param N/A
Multicast Flow Control Strategy Supplier (C Driver)
Description Supplier of UDP multicast flow control strategies. Should be one of multicast_min, multicast_max, multicast_tagged or the name of a supplier function to dynamically load.
Type (property) const char *
Type (API) aeron_flow_control_strategy_supplier_func_t
Default "aeron_max_multicast_flow_control_strategy_supplier"
System Property aeron.multicast.FlowControl.supplier
Context aeron_driver_context_set_multicast_flowcontrol_supplier()
URI Param N/A
Congestion Control Supplier (Java Driver)
Description Supplier of dynamically created CongestionControl strategies for individual connections.
Type io.aeron.driver.CongestionControlSupplier
Default io.aeron.driver.DefaultCongestionControlSupplier
System Property aeron.CongestionControl.supplier
Context MediaDriver.Context.congestionControlSupplier()
URI Param N/A
Congestion Control Supplier (C Driver)
Description Supplier of dynamically created CongestionControl strategies for individual connections. Should be name of a supplier function to dynamically load.
Type (property) const char *
Type (API) aeron_congestion_control_strategy_supplier_func_t
Default "aeron_static_window_congestion_control_strategy_supplier"
System Property aeron.CongestionControl.supplier
Context aeron_driver_context_set_congestioncontrol_supplier()
URI Param N/A
MTU (Maximum Transmission Unit) Length
Description Maximum length of a message fragment including Aeron data frame header for transmission in a network packet. This can be a larger than an Ethernet MTU provided it is smaller than the maximum UDP payload length. Larger lengths enable batching and reducing syscalls at the expense of more likely loss.
Type (Java / C) int32 / size_t
Default 1408
System Property aeron.mtu.length
Context MediaDriver.Context.mtuLength()
URI Param mtu
IPC MTU (Maximum Transmission Unit) Length
Description Maximum length of a message fragment including Aeron data frame header for transmission over IPC. This cannot be greater than an network MTU as the stream can be recorded locally and replayed remote.
Type (Java / C) int32 / size_t
Default 1408
System Property aeron.ipc.mtu.length
Context MediaDriver.Context.ipcMtuLength()
URI Param mtu
Threading Mode for the Driver
Description DEDICATED is a thread for each of the Conductor, Sender, and Receiver Agents. SHARED is one thread for all three Agents. SHARED_NETWORK is one thread for the conductor and one shared for the Sender and Receiver Agents. INVOKER is a mode with no threads, i.e. the client is responsible for using the MediaDriver.Context.driverAgentInvoker() to invoke the duty cycle directly. For embedding the C driver prefix the name with AERON_THREADING_MODE_.
Type (Java / C) ThreadingMode / aeron_threading_mode_t
Default DEDICATED
System Property aeron.threading.mode
Context MediaDriver.Context.threadingMode()
URI Param N/A
Asynchronous Executor Threads
Description Configure the number of asynchronous executor threads (used for async DNS name resolution). Setting this property to zero or negative value disables async execution.
Type int32
Default 1
System Property aeron.driver.async.executor.threads
Context MediaDriver.Context#asyncTaskExecutorThreads()
URI Param N/A
Since Version 1.44.0
Send to Status Poll Ratio
Description Ratio of maximum number of data packets sent per publication to poll actions for control messages.
Type (Java / C) int32 / size_t
Default 4
System Property aeron.send.to.status.poll.ratio
Context MediaDriver.Context.sendToStatusMessagePollRatio()
URI Param N/A
Spy Simulates Connection
Description Does a spy subscription simulate a connection to a network publication.
Type boolean
Default false
System Property aeron.spies.simulate.connection
Context MediaDriver.Context.spiesSimulateConnection()
URI Param N/A
Reliable Stream
Description Does a stream NAK when loss is detected, reliable=true, or gap fill, reliable=false.
Type boolean
Default true
System Property aeron.reliable.stream
Context MediaDriver.Context.reliableStream()
URI Param reliable
Tether Subscriptions
Description Do subscriptions have a tether so they participates in local flow control when more than one.
Type boolean
Default true
System Property aeron.tether.subscriptions
Context MediaDriver.Context.tetherSubscriptions()
URI Param tether
Receiver Group Consideration
Description Should network subscriptions be considered part of a group even if using a unicast endpoint, should it be considered an individual even if using a multicast endpoint, or should the use of a unicast/multicast endpoint infer the usage.
Type (Java / C) io.aeron.CommonContext.InferableBoolean / aeron_inferable_boolean_t
Default INFER
System Property aeron.receiver.group.consideration
Context MediaDriver.Context.receiverGroupConsideration()
URI Param group
Rejoin Stream
Description Does a subscription attempt to rejoin an unavailable stream after a cooldown or not.
Type boolean
Default true
System Property aeron.rejoin.stream
Context MediaDriver.Context.rejoinStream()
URI Param rejoin
Epoch Clock (Java Only)
Description The EpochClock to be used as a source of time in milliseconds for wall clock time.
Type org.agrona.concurrent.EpochClock
Default org.agrona.concurrent.SystemEpochClock
System Property N/A
Context MediaDriver.Context.epochClock()
URI Param N/A
Nano Clock (Java Only)
Description The NanoClock to be used as a source of time in nanoseconds for measuring duration.
Type org.agrona.concurrent.NanoClock
Default org.agrona.concurrent.SystemNanoClock
System Property N/A
Context MediaDriver.Context.nanoClock()
URI Param N/A
Cached Epoch Clock (Java Only)
Description The CachedEpochClock to be used as a source of time in milliseconds for wall clock time.
Type org.agrona.concurrent.CachedEpochClock
Default org.agrona.concurrent.CachedEpochClock
System Property N/A
Context MediaDriver.Context.cachedEpochClock()
URI Param N/A
Cached Nano Clock (Java Only)
Description The CachedNanoClock to be used as a source of time in nanoseconds for measuring duration. This is updated once per duty cycle of the DriverConductor.
Type org.agrona.concurrent.CachedNanoClock
Default org.agrona.concurrent.CachedNanoClock
System Property N/A
Context MediaDriver.Context.cachedNanoClock()
URI Param N/A
Sender Thread Factory (Java Only)
Description ThreadFactory to be used for creating agent thread for the Sender when running in ThreadingMode#DEDICATED.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context MediaDriver.Context.senderThreadFactory()
URI Param N/A
Receiver Thread Factory (Java Only)
Description ThreadFactory to be used for creating agent thread for the Receiver when running in ThreadingMode#DEDICATED.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context MediaDriver.Context.conductorThreadFactory()
URI Param N/A
Conductor Thread Factory (Java Only)
Description ThreadFactory to be used for creating agent thread for the DriverConductor when running in ThreadingMode#DEDICATED or ThreadingMode#SHARED_NETWORK.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context MediaDriver.Context.conductorThreadFactory()
URI Param N/A
Shared Thread Factory (Java Only)
Description ThreadFactory to be used for creating agent thread for the all driver agents as a CompositeAgent when running in ThreadingMode#SHARED.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context MediaDriver.Context.sharedThreadFactory()
URI Param N/A
Shared Network Thread Factory (Java Only)
Description ThreadFactory to be used for creating agent thread for the sender and receiver agents as a CompositeAgent when running in ThreadingMode#SHARED_NETWORK.
Type java.util.concurrent.ThreadFactory
Default (Runnable task) -> new java.lang.Thread(task)
System Property N/A
Context MediaDriver.Context.sharedNetworkThreadFactory()
URI Param N/A
Temp Buffer (Java Only)
Description The temporary buffer than can be used to build up counter labels to avoid allocation.
Type org.agrona.MutableDirectBuffer
Default new UnsafeBuffer(new byte[METADATA_LENGTH])
System Property N/A
Context MediaDriver.Context.tempBuffer()
URI Param N/A
Application Specific Feedback (deprecated) (Java Only)
Description Use 'Receiver Group Tag' instead.
Type byte[]
Default new byte[0]
System Property aeron.flow.control.sm.applicationSpecificFeedback
Context MediaDriver.Context.applicationSpecificFeedback()
URI Param N/A
Receiver Group Tag
Description Group tag to be sent back to the senders on the Status Message from receivers.
Type (Java / C) Long / (bool & int64_t)
Default (Java / C) null / (false, -1)
System Property aeron.receiver.group.tag
Context MediaDriver.Context.receiverGroupTag()
URI Param gtag
Flow Control Group Tag
Description The group tag to used on the Publication side by the tagged flow control strategy.
Type int64
Default -1
System Property aeron.flow.control.group.tag
Context MediaDriver.Context.flowControlGroupTag()
URI Param fc
Flow Control Receiver Timeout
Description Timeout (ns) for receivers in flow control that haven't sent status messages.
Type (Java / C) int64 / uint64_t
Default 2_000_000_000 (2 seconds)
System Property aeron.flow.control.receiver.timeout
Context MediaDriver.Context.flowControlReceiverTimeoutNs()
URI Param fc
Flow Control Group Minimum Size
Description Minimum size for a group within the min or tagged flow control strategy to determine if a Publication can be connected.
Type int32
Default 0
System Property aeron.flow.control.group.min.size
Context MediaDriver.Context.flowControlGroupMinSize()
URI Param fc
Use Concurrent Counters Manager (Java Only)
Description Should a ConcurrentCountersManager be used to allow for cross thread usage.
Type boolean
Default false
System Property N/A
Context MediaDriver.Context.useConcurrentCountersManager()
URI Param N/A
Counters Manager (Java Only)
Description Set the CountersManager to override the one that would have been concluded.
Type org.agrona.concurrent.status.CountersManager
Default null
System Property N/A
Context MediaDriver.Context.countersManager()
URI Param N/A
System Counters (Java Only)
Description The SystemCounters for the driver for recording aggregate events of system status.
Type io.aeron.driver.status.SystemCounters
Default io.aeron.driver.status.SystemCounters
System Property N/A
Context MediaDriver.Context.systemCounters()
URI Param N/A
Publication Reserved Session Id Low
Description Low end of the publication reserved session id range which will not be automatically assigned.
Type int32
Default -1
System Property aeron.publication.reserved.session.id.low
Context MediaDriver.Context.publicationReservedSessionIdLow()
URI Param N/A
Publication Reserved Session Id High
Description High end of the publication reserved session id range which will not be automatically assigned.
Type int32
Default 1000
System Property aeron.publication.reserved.session.id.high
Context MediaDriver.Context.publicationReservedSessionIdHigh()
URI Param N/A
Termination Hook (Java Driver)
Description Callback that is called when the MediaDriver processes a valid termination request.
Type java.lang.Runnable
Default null
System Property N/A
Context MediaDriver.Context.terminationHook()
URI Param N/A
Termination Hook (C Driver)
Description Callback that is called when the MediaDriver processes a valid termination request.
Type aeron_driver_termination_hook_func_t
Default null
System Property N/A
Context aeron_driver_context_set_driver_termination_hook()
URI Param N/A
Termination Validator (Java Driver)
Description TerminationValidator to be used to validate termination requests.
Type io.aeron.driver.TerminationValidator
Default io.aeron.driver.DefaultDenyTerminationValidator
System Property aeron.driver.termination.validator
Context MediaDriver.Context.terminationValidator()
URI Param N/A
Termination Validator (C Driver)
Description TerminationValidator to be used to validate termination requests. String of allow or deny
Type (property) const char *
Type (API) aeron_driver_termination_validator_func_t
Default "deny"
System Property aeron.driver.termination.validator
Context aeron_driver_context_set_driver_termination_validator()
URI Param N/A
Re-resolution check interval
Description Frequency that names should be checked for re-resolution.
Type (Java / C) int64 / uint64_t
Default 1_000_000_000 (1s)
System Property aeron.driver.reresolution.check.interval
Context MediaDriver.Context.reResolutionCheckIntervalNs()
URI Param N/A
Driver Resolver Name
Description Name for the driver resolver.
Type (Java) String
Default Canonical name of the host.
System Property aeron.driver.resolver.name
Context MediaDriver.Context.resolverName()
URI Param N/A
Driver Resolver Interface
Description Interface for the driver resolver in the form address:port/mask.
Type (Java) String
Default null
System Property aeron.driver.resolver.interface
Context MediaDriver.Context.resolverInterface()
URI Param N/A
Driver Resolver Bootstrap Neighbor
Description Bootstrap neighbor for the driver resolver in the form host:port.
Type (Java) String
Default null
System Property aeron.driver.resolver.bootstrap.neighbor
Context MediaDriver.Context.resolverBootstrapNeighbor()
URI Param N/A
Resource Free Limit
Description Maximum number of DriverManagedResource(s) to free within a single duty cycle within the conductor.
Type (Java/C) int/uint32_t
Default 10
System Property aeron.driver.resource.free.limit
Context MediaDriver.Context.resourceFreeLimit()
URI Param N/A
Since Version 1.41.0
Conductor Thread Affinity (C only)
Description Set the CPU to pin the conductor thread to. Only works if launching the aeronmd binary. Embedded users will need to implement this themselves.
Type (C) int32_t
Default -1
System Property aeron.conductor.cpu.affinity
Context N/A
URI Param N/A
Sender Thread Affinity (C only)
Description Set the CPU to pin the sender thread to. Only works if launching the aeronmd binary. Embedded users will need to implement this themselves.
Type (C) int32_t
Default -1
System Property aeron.sender.cpu.affinity
Context N/A
URI Param N/A
Receiver Thread Affinity (C only)
Description Set the CPU to pin the receiver thread to. Only works if launching the aeronmd binary. Embedded users will need to implement this themselves.
Type (C) int32_t
Default -1
System Property aeron.receiver.cpu.affinity
Context N/A
URI Param N/A
Socket Connect (C only)
Description Set the UDP transport to use connect and send instead of sendmsg to potentially reduce system call cost.
Type (C) bool
Default true
System Property aeron.driver.connect
Context aeron_driver_context_set_connect_enabled
URI Param N/A
Since Version 1.43.0
Stream Session Limit
Description Limit the maximum number of sessions allowed per stream for a subscription.
Type (Java/C) int/int32_t
Default 2147483647 (Max signed 32 bit int, effectively unlimited)
System Property aeron.driver.stream.session.limit
Context MediaDriver.Context.streamSessionLimit()
URI Param N/A
Since Version 1.45.0

Event Logging Options

Debug Logging can be enabled in the driver via a Java agent which performs bytecode weaving. Aeron pays zero penalty for logging when it is not enabled.

Event Buffer Length
Description Length of the ring buffer into which events are captured for hand off to the logger.
Type uint32
Default 8 * 1024 * 1024
System Property aeron.event.buffer.length
Context N/A
URI Param N/A
Event Log Reader (Agent)
Description Reads the event buffer and outputs to console, file, or other location.
Type String
Default io.aeron.agent.EventLogReaderAgent
System Property aeron.event.log.reader.classname
Context N/A
URI Param N/A
Event Log Filename
Description Optional filename that the EventLogReaderAgent will output to is set, otherwise STDOUT is used.
Type String
Default null
System Property aeron.event.log.filename
Context N/A
URI Param N/A
Driver Enabled Event Codes
Description Comma separated list of event codes that should be logged from the DriverEventCode enum.
Type String
Default null
System Property aeron.event.log
Context N/A
URI Param N/A
Archive Enabled Event Codes
Description Comma separated list of event codes that should be logged from the ArchiveEventCode enum.
Type String
Default null
System Property aeron.event.archive.log
Context N/A
URI Param N/A
Cluster Enabled Event Codes
Description Comma separated list of event codes that should be logged from the ClusterEventCode enum.
Type String
Default null
System Property aeron.event.cluster.log
Context N/A
URI Param N/A

Channel Endpoint Extension Options (Java Driver)

Aeron is designed for extension. One good example is how the interaction with the network media layer can be extended for interaction on the sender or receiver side. Channel endpoints can sub-classed by configured suppliers. Examples are provided for testing and debugging that introduce packet loss.

Send Channel Endpoint Supplier
Description Supplier of custom behaviour for SendChanneEndpoints to override interaction with the network.
Type io.aeron.driver.SendChannelEndpointSupplier
Default io.aeron.driver.DefaultSendChannelEndpointSupplier
System Property aeron.SendChannelEndpoint.supplier
Context MediaDriver.Context.sendChannelEndpointSupplier()
URI Param N/A
Receive Channel Endpoint Supplier
Description Supplier of custom behaviour for ReceiveChanneEndpoints to override interaction with the network.
Type io.aeron.driver.ReceiveChannelEndpointSupplier
Default io.aeron.driver.DefaultReceiveChannelEndpointSupplier
System Property aeron.ReceiveChannelEndpoint.supplier
Context MediaDriver.Context.receiveChannelEndpointSupplier()
URI Param N/A
Receive Channel Thread-local Buffers
Description The thread local buffers and associated objects for use by subclasses of ReceiveChannelEndpoint.
Type io.aeron.driver.media.ReceiveChannelEndpointThreadLocals
Default io.aeron.driver.media.ReceiveChannelEndpointThreadLocals
System Property N/A
Context MediaDriver.Context.receiveChannelEndpointThreadLocals()
URI Param N/A
Send Data Loss Rate
Description Factor for the rate at which data packets should be discarded to simulate loss on send end of channel.
Type double
Default 0.0
System Property aeron.debug.send.data.loss.rate
Context N/A
URI Param N/A
Send Data Loss Seed
Description Seed for the random number generator for data packet loss.
Type int64
Default -1
System Property aeron.debug.send.data.loss.seed
Context N/A
URI Param N/A
Send Control Loss Rate
Description Factor for the rate at which control packets should be discarded to simulate loss on send end of channel.
Type double
Default 0.0
System Property aeron.debug.send.control.loss.rate
Context N/A
URI Param N/A
Send Control Loss Seed
Description Seed for the random number generator for control packet loss.
Type int64
Default -1
System Property aeron.debug.send.control.loss.seed
Context N/A
URI Param N/A
Receive Data Loss Rate
Description Factor for the rate at which data packets should be discarded to simulate loss on receive end of channel.
Type double
Default 0.0
System Property aeron.debug.receive.data.loss.rate
Context N/A
URI Param N/A
Receive Data Loss Seed
Description Seed for the random number generator for data packet loss.
Type int64
Default -1
System Property aeron.debug.receive.data.loss.seed
Context N/A
URI Param N/A
Receive Control Loss Rate
Description Factor for the rate at which control packets should be discarded to simulate loss on receive end of channel.
Type double
Default 0.0
System Property aeron.debug.receive.control.loss.rate
Context N/A
URI Param N/A
Receive Control Loss Seed
Description Seed for the random number generator for control packet loss.
Type int64
Default -1
System Property aeron.debug.receive.control.loss.seed
Context N/A
URI Param N/A

UDP Channel Transport Bindings (C Driver)

The C driver manages extensions to the driver in a different manner to the Java implementation and uses the concept of UDP transport bindings and interceptors to provide similar functionality.

UDP Channel Transport Bindings
Description The transport bindings for sending and receiving UDP network traffic.
Type (property) const char *
Type (API) aeron_udp_channel_transport_bindings_t
Default "default"
System Property aeron.udp.channel.transport.bindings.media
Context aeron_driver_context_set_udp_channel_transport_bindings()
URI Param N/A
UDP Channel Outgoing Interceptors
Description List of interceptors for outgoing network traffic, e.g logging for packet logging.
Type (property) const char *
Type (API) aeron_udp_channel_interceptor_bindings_t
Default NULL
System Property aeron.udp.channel.outgoing.interceptors
Context aeron_driver_context_set_udp_channel_outgoing_interceptors()
URI Param N/A
UDP Channel Incoming Interceptors
Description List of interceptors for incoming network traffic, e.g loss for configuring loss generation.
Type (property) const char *
Type (API) aeron_udp_channel_interceptor_bindings_t
Default NULL
System Property aeron.udp.channel.incoming.interceptors
Context aeron_driver_context_set_udp_channel_incoming_interceptors()
URI Param N/A

Aeron Archive Client Options

Control Response Channel
Description Channel for receiving control response messages from an archive.
Type String
Default None, must be user supplied
System Property aeron.archive.control.response.channel
Context AeronArchive.Context.controlResponseChannel()
URI Param N/A

Starting with version 1.30.0, the aeron.archive.control.response.channel can be configured without specifying an explicit port number. The following list shows examples of valid configurations:

  • aeron:udp?endpoint=localhost:8020 - listen on port 8020 via localhost.
  • aeron:udp?endpoint=192.168.10.10:8020 - listen on port 8020 via 192.168.10.10.
  • aeron:udp?endpoint=localhost:0 - in this case the port is unspecified and the OS will assign a free port from the ephemeral port range.

Aeron Cluster Client Options

Egress Channel
Description Channel for receiving response messages from a cluster.
Type String
Default aeron:udp?endpoint=localhost:0
System Property aeron.cluster.egress.channel
Context AeronCluster.Context.egressChannel()
URI Param N/A

Starting with version 1.30.0, the aeron.cluster.egress.channel can be configured without specifying an explicit port number. The following list shows examples of valid configurations:

  • aeron:udp?endpoint=localhost:9020 - listen on port 9020 via localhost.
  • aeron:udp?endpoint=192.168.10.10:9020 - listen on port 9020 via 192.168.10.10.
  • aeron:udp?endpoint=localhost:0 - in this case the port is unspecified and the OS will assign a free port from the ephemeral port range.