Apache Ignite Documentation

GridGain Developer Hub - Apache Ignitetm

Welcome to the Apache Ignite developer hub run by GridGain. Here you'll find comprehensive guides and documentation to help you start working with Apache Ignite as quickly as possible, as well as support if you get stuck.

 

GridGain also provides Community Edition which is a distribution of Apache Ignite made available by GridGain. It is the fastest and easiest way to get started with Apache Ignite. The Community Edition is generally more stable than the Apache Ignite release available from the Apache Ignite website and may contain extra bug fixes and features that have not made it yet into the release on the Apache website.

 

Let's jump right in!

 

Documentation     Ask a Question     Download

 

Javadoc     Scaladoc     Examples

Transactions

ACID compliant transactions ensuring guaranteed consistency.

Atomicity Mode

Ignite supports two modes for cache operations, transactional and atomic. In the TRANSACTIONAL mode, you are able to group multiple cache operations in a transaction, whilst the ATOMIC mode supports multiple atomic operations, one at a time.

The TRANSACTIONAL mode enables fully ACID-compliant transactions. In this mode, you are able to group multiple cache operations, on one or more keys, in to a single logical operation, known as a transaction. These operations will be executed without any other interleaved operations on the specified keys, and will either all succeed or all fail. There is no partial execution of the operations. For better performance, use the ATOMIC mode. Only enable TRANSACTIONAL mode if you require ACID-compliant operation.

The ATOMIC mode provides better performance by avoiding transactional locks, whilst still providing data atomicity and consistency for each single operation. Another difference in ATOMIC mode is that bulk writes, such as PutAll(...)and RemoveAll(...) methods are not executed in one transaction and can partially fail. If this partial failure occurs, a CachePartialUpdateException will be thrown and will contain a list of keys for which the update failed.

Atomicity mode is defined in CacheAtomicityMode enum and can be configured via the atomicityMode property of CacheConfiguration.

<bean class="org.apache.ignite.configuration.IgniteConfiguration">
    ...
    <property name="cacheConfiguration">
        <bean class="org.apache.ignite.configuration.CacheConfiguration">
            <!-- Set a cache name. -->
            <property name="name" value="myCache"/>

            <!-- Set atomicity mode, can be ATOMIC or TRANSACTIONAL. 
                ATOMIC is default. -->
            <property name="atomicityMode" value="TRANSACTIONAL"/>
            ... 
        </bean>
    </property>
     
    <!-- Optional transaction configuration. -->
    <property name="transactionConfiguration">
        <bean class="org.apache.ignite.configuration.TransactionConfiguration">
            <!-- Configure TM lookup here. -->
        </bean>
    </property>
</bean>
CacheConfiguration cacheCfg = new CacheConfiguration();

cacheCfg.setName("cacheName");

cacheCfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);

IgniteConfiguration cfg = new IgniteConfiguration();

cfg.setCacheConfiguration(cacheCfg);

// Optional transaction configuration. Configure TM lookup here.
TransactionConfiguration txCfg = new TransactionConfiguration();

cfg.setTransactionConfiguration(txCfg);

// Start Ignite node.
Ignition.start(cfg);

Performance

Note that transactions are disabled whenever ATOMIC mode is used, which allows to achieve much higher performance and throughput in cases when transactions are not needed.

IgniteTransactions

IgniteTransactions interface contains functionality for starting and completing transactions, as well as subscribing listeners or getting metrics.

Cross-Cache Transactions

You can combine multiple operations from different caches into one transaction. Note that this allows you to update caches of different types, like REPLICATED and PARTITIONED caches, in one transaction.

Near Cache Transactions

Near caches are fully transactional and get updated or invalidated automatically whenever the data changes on the servers.

You can obtain an instance of IgniteTransactions as follows:

Ignite ignite = Ignition.ignite();

IgniteTransactions transactions = ignite.transactions();

Here is an example of how transactions can be performed in Ignite:

try (Transaction tx = transactions.txStart()) {
    Integer hello = cache.get("Hello");
  
    if (hello == 1)
        cache.put("Hello", 11);
  
    cache.put("World", 22);
  
    tx.commit();
}

Transaction Object Lifecycle

A Transaction object needs to be closed after the usage. To ensure this always happens, do one of the following:

  • Start the Transaction in a try-with-resources statement that calls close() method in the end.
  • Use the finally block and call tx.close() method manually.

Transactional Methods

Not all of the methods in the IgniteCache API are fully transactional when this mode is enabled for a cache. Methods that have "throws TransactionException" in its method signature satisfy the ACID principle and can be safely used inside of a distributed transaction.

Two-Phase-Commit (2PC)

Ignite utilizes a Two-Phase-Commit (2PC) protocol for its transactions and optimizes to one-phase-commit whenever possible. Whenever data is updated within a transaction, Ignite will keep a transactional state in a local transaction map until commit() is called, at which point, if needed, the data is transferred to participating remote nodes.

As the name implies, there are two phases: prepare and commit that are explained in the following blog posts.

Original series:

New series that takes Ignite persistence into consideration:

Also​, check this page that covers internals of Ignite's transactional subsystem.

ACID Compliance

Ignite provides fully ACID (Atomicity, Consistency, Isolation, Durability) compliant transactions that ensure guaranteed consistency.

Concurrency Modes and Isolation Levels

Whenever TRANSACTIONAL atomicity mode is configured, Ignite supports OPTIMISTIC and PESSIMISTIC concurrency modes for transactions. Concurrency mode determines when an entry-level transaction lock should be acquired - at the time of data access or during the prepare phase. Locking prevents concurrent access to an object. For example, when you attempt to update a ToDo list item with pessimistic locking, the server places a lock on the object until you either commit or rollback the transaction so that no other transaction or operation is allowed to update the same entry. Regardless of the concurrency mode used in a transaction, there exists a moment in time when all entries enlisted in the transaction are locked before the commit.
Isolation level defines how concurrent transactions will 'see' and handle operations on the same keys. Ignite supports READ_COMMITTED, REPEATABLE_READ and SERIALIZABLE isolation levels.
All combinations of concurrency modes and isolation levels can be used simultaneously. Below is the description of Ignite behavior and guarantees provided by each concurrency-isolation combination.

Pessimistic Transactions

In PESSIMISTIC transactions, locks are acquired during the first read or write access (depending on the isolation level) and held by the transaction until it is committed or rolled back. In this mode locks are acquired on primary nodes first and then promoted to backup nodes during the prepare stage. The following isolation levels can be configured with PESSIMISTIC concurrency mode:

  • READ_COMMITTED - Data is read without a lock and is never cached in the transaction itself. The data may be read from a backup node if this is allowed in the cache configuration. In this isolation you can have the so-called Non-Repeatable Reads because a concurrent transaction can change the data when you are reading the data twice in your transaction. The lock is only acquired at the time of first write access (this includes EntryProcessor invocation). This means that an entry that have been read during the transaction may have a different value by the time the transaction is committed. No exception will be thrown in this case.

  • REPEATABLE_READ - Entry lock is acquired and data is fetched from the primary node on the first read or write access and stored in the local transactional map. All consecutive access to the same data is local and will return the last read or updated transaction value. This means no other concurrent transactions can make changes to the locked data, and you are getting Repeatable Reads for your transaction.

  • SERIALIZABLE - In PESSIMISTIC mode, this isolation level works the same way as REPEATABLE_READ.

Note that in PESSIMISTIC mode, the order of locking is important. Moreover, Ignite will acquire locks sequentially and exactly in the order provided by a user.

Performance Considerations

Imagine that you have 3 nodes in your topology (A, B, C) and in your transaction you are doing a putAll for keys [1, 2, 3, 4, 5, 6]. Suppose that these keys are mapped to nodes in the following fashion: {A: 1, 4}, {B: 2, 5}, {C: 3, 6}. Since Ignite cannot re-arrange the lock acquisition order in PESSIMISTIC mode, it will have to make 6 sequential network round-trips: [A, B, C, A, B, C]. In a case when the key locking order is not important for the semantics of a transaction, it is advisable to group keys by partition and lock keys within the same partition together. This may significantly reduce the number of network messages in a large transaction. In this example, if keys were ordered for a putAll in the following way: [1, 4, 2, 5, 3, 6], then only 3 sequential round-trips would be required.

Topology Change Restrictions

Note that if at least one PESSIMISTIC transaction lock is acquired, it will be impossible to change the cache topology until the transaction is committed or rolled back. Therefore, it is not recommended to hold transaction locks for a long period of time.

Optimistic Transactions

In OPTIMISTIC transactions, entry locks are acquired on primary nodes during the first phase of 2PC, at prepare step, then promoted to backup nodes and released once the transaction is committed. The locks are never acquired if the transaction is rolled back by user and no commit attempt was made. The following isolation levels can be configured with OPTIMISTIC concurrency mode:

  • READ_COMMITTED - Changes that should be applied to the cache are collected on the originating node and applied upon the transaction commit. Transaction data is read without a lock and is never cached in the transaction. The data may be read from a backup node if this is allowed in the cache configuration. In this isolation you can have so-called Non-Repeatable Reads because a concurrent transaction can change the data when you are reading the data twice in your transaction. This mode combination does not check if the entry value has been modified since the first read or write access and never raises an optimistic exception.

  • REPEATABLE_READ - Transactions at this isolation level work similar to OPTIMISTIC READ_COMMITTED transactions with only one difference - read values are cached on the originating node and all subsequent reads are guaranteed to be local. This mode combination does not check if the entry value has been modified since the first read or write access and never raises an optimistic exception.

  • SERIALIZABLE - Stores an entry version upon first read access. Ignite will fail a transaction at the commit stage if the Ignite engine detects that at least one of the entries used as part of the initiated transaction has been modified. This is achieved by internally checking the version of an entry remembered in a transaction to the one actually in the grid at the time of commit. In short, this means that if Ignite detects that there is a conflict at the commit stage of a transaction, we fail such a transaction throwing TransactionOptimisticException & rolling back any changes made. User should handle this exception and retry the transaction.

// Re-try transaction finite amount of time.
int retryCount = 10;
int retries = 0;

// Start transaction in optimistic mode with serializable isolation level.
while (retries < retryCount) {
    retries++;
    try (Transaction tx =  
        ignite.transactions().txStart(TransactionConcurrency.OPTIMISTIC,
                                      TransactionIsolation.SERIALIZABLE)) {
        // Modify cache entries as part of this transaction.
        ....
        
        // commit transaction.  
        tx.commit();

        // Transaction succeeded. Leave the while loop.
        break;
    }
    catch (TransactionOptimisticException e) {
        // Transaction has failed. Retry.
    }
}

Another important point to note here is that a transaction will still fail even if an entry that was simply read (with no modify, cache.put(...)) since the value of the entry could be important to the logic within the initiated transaction.

Note that the key order is important for READ_COMMITTED and REPEATABLE_READ transactions since the locks are still acquired sequentially in these modes.

Retrying Optimistic Serializable Transactions

It is always a good idea to retry a failed optimistic serializable transaction because the data, the transaction was trying to update, might have been changed concurrently. Therefore, the TransactionOptimisticException should be handled with retry logic. It is reasonable to constraint the retry count with some limit.

Deadlock Detection

One major rule that anyone has to follow when working with distributed transactions is that locks for keys, participating in a transaction, must be acquired in the same order. Violating this rule may lead to a distributed deadlock.

Ignite does not avoid distributed deadlocks, but rather has a built-in functionality that makes it easier to debug and fix such situations.

As shown in the code snippet below, a transaction has been started with a timeout. If the timeout expires, the deadlock detection procedure will try to find a possible deadlock that might have caused the timeout. When the timeout expires, TransactionTimeoutException is generated and propagated to the application code as the cause of CacheException regardless of a deadlock. However, if a deadlock is detected, the cause of the returned TransactionTimeoutException will be TransactionDeadlockException (at least for one transaction involved in the deadlock).

try (Transaction tx = ignite.transactions().txStart(TransactionConcurrency.PESSIMISTIC,
    TransactionIsolation.READ_COMMITTED, 300, 0)) {
    cache.put(1, 1);

    cache.put(2, 1);

    tx.commit();
}
catch (CacheException e) {
    if (e.getCause() instanceof TransactionTimeoutException &&
        e.getCause().getCause() instanceof TransactionDeadlockException)    
        
        System.out.println(e.getCause().getCause().getMessage());
}

TransactionDeadlockException message contains useful information that can help you find the reason for the deadlock.

Deadlock detected:

K1: TX1 holds lock, TX2 waits lock.
K2: TX2 holds lock, TX1 waits lock.

Transactions:

TX1 [txId=GridCacheVersion [topVer=74949328, time=1463469328421, order=1463469326211, nodeOrder=1], nodeId=ad68354d-07b8-4be5-85bb-f5f2362fbb88, threadId=73]
TX2 [txId=GridCacheVersion [topVer=74949328, time=1463469328421, order=1463469326210, nodeOrder=1], nodeId=ad68354d-07b8-4be5-85bb-f5f2362fbb88, threadId=74]

Keys:

K1 [key=1, cache=default]
K2 [key=2, cache=default]

Deadlock detection is a multi step procedure that may take many iterations depending on the number of nodes in the cluster, keys, and transactions that are involved in a possible deadlock. A deadlock detection initiator is a node where a transaction was started and failed with a TransactionTimeoutException. This node will investigate if a deadlock has occurred, by exchanging requests/responses with other remote nodes, and prepare a deadlock related report that is provided with the TransactionDeadlockException. Each such message (request/response) is known as an iteration.

Since a transaction is not rolled back until the deadlock detection procedure is completed, sometimes, it makes sense to tune the parameters (shown below), if you want to have a predictable time for a transaction's rollback.

  • IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS - Specifies the maximum number of iterations for the deadlock detection procedure. If the value of this property is less than or equal to zero, the deadlock detection will be disabled (1000 by default);
  • IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_TIMEOUT - Specifies timeout for the deadlock detection mechanism (1 minute by default).

Note that if there are too few iterations, you may get an incomplete deadlock-report.

If you want to completely avoid deadlocks, refer to Deadlock-free Transactions section below.

Deadlock-Free Transactions

For OPTIMISTIC SERIALIZABLE transactions, locks are not acquired sequentially. In this mode, keys can be accessed in any order because transaction locks are acquired in parallel with an additional check allowing Ignite to avoid deadlocks.
We need to introduce some concepts in order to describe how locks in SERIALIZABLE transactions work. In Ignite, each transaction is assigned a comparable version called XidVersion. Upon transaction commit, each entry that is written in the transaction is assigned a new comparable version called EntryVersion. An OPTIMISTIC SERIALIZABLE transaction with version XidVersionA will fail with a TransactionOptimisticException if:

  • There is an ongoing PESSIMISTIC or non-serializable OPTIMISTIC transaction holding a lock on an entry of the SERIALIZABLE transaction.
  • There is another ongoing OPTIMISTIC SERIALIZABLE transaction with version XidVersionB such that XidVersionB > XidVersionA and this transaction holds a lock on an entry of the SERIALIZABLE transaction.
  • By the time the OPTIMISTIC SERIALIZABLE transaction acquires all required locks, there exists an entry with the current version different from the observed version before commit.

In a highly concurrent environment, optimistic locking might lead to a high transaction failure rate but pessimistic locking can lead to deadlocks if locks are acquired in a different order by transactions.
However, in a contention-free environment optimistic serializable locking may provide better performance for large transactions because the number of network trips depends only on the number of nodes that the transaction spans and does not depend on the number of keys in the transaction.

Handling Failed Transactions

A transaction might fail with the following exceptions:

Exception name
Description

CacheException caused by TransactionTimeoutException

TransactionTimeoutException is generated if the transaction times out.


Solution: To solve this exception, increase the timeout or make the transaction shorter.


CacheException caused by TransactionTimeoutException,which is caused by TransactionDeadlockException


This exception is generated if the transaction goes into a deadlock.


Solution: Use the deadlock detection mechanism to debug and fix the deadlock or switch to the optimistic serializable transactions (deadlock-free transactions).


TransactionOptimisticException


This exception is thrown if the optimistic transaction fails for some reason. In most of the scenarios, this exception occurs when the data, the transaction was trying to update, was changed concurrently.


Solution: To solve this exception, you will need to rerun the transaction.


TransactionRollbackException


This exception occurs if the transaction was rolled back automatically or manually. In this case, the data state is consistent.


Solution: Since the data is in a consistent state, you can retry the transaction.


TransactionHeuristicException


That's an unlikely exception that happens due to an unexpected internal or communication issue in Ignite. The exception exists to report problematic scenarios that were not foreseen by the transactional subsystem and were not handled by it properly.



Solution: The data might not stay consistent if the exception occurs. Reload the data and report to Ignite development community.

Retrying Transactions

It's reasonable to retry both failed optimistic and pessimistic transactions. Even if there is a failed transaction due to some issue caused by network problems or node failures, Ignite will use backup copies or the data available on disk to enforce data consistency. All this allows to re-try a transaction if needed.

However, don't try to re-run the transaction infinitely on the application level. Include​ a logic that does this for a finite amount of time.

Long Running Transactions Termination

Some cluster events trigger partition map exchange process and data rebalancing within an Ignite cluster to ensure even data distribution cluster-wide. An example of one such event is cluster-topology-change event that takes place whenever a new node joins the cluster, or an existing one leaves it. Plus, every time a new cache or SQL table is created, the partition map exchange gets triggered.

When the partition map exchange starts, Ignite acquires a global lock at a particular stage. The lock can't be obtained until incomplete transactions are running in parallel. These transactions prevent the partition map exchange process from moving forward​, thus, blocking some operations such as a new node join process.

Use TransactionConfiguration.setTxTimeoutOnPartitionMapExchange(...) method to set the maximum time allowed for your long-running transactions to block the partition map exchange. Once the timeout fires, all the incomplete transactions will be rolled back letting the partition map exchange to proceed.

This example shows how to configure the timeout:

<bean class="org.apache.ignite.configuration.IgniteConfiguration">
  	...
    <property name="transactionConfiguration">
        <bean class="org.apache.ignite.configuration.TransactionConfiguration">
        <!--Set the timeout to 20 seconds-->
        <property name="TxTimeoutOnPartitionMapExchange" value="20000"/>
        <!--Other trasaction configurations-->
        ...
        </bean>
    </property>
</bean>
// Create Ignite configuration
IgniteConfiguration cfg = new IgniteConfiguration();
        
// Create Ignite Transactions configuration
TransactionConfiguration txCfg = new TransactionConfiguration();

// Set the timeout to 20 seconds
txCfg.setTxTimeoutOnPartitionMapExchange(20000);

cfg.setTransactionConfiguration(txCfg);

// Start the cluster node
Ignition.start(cfg);

If a transaction is rolled back due to the timeout fired, you'll be able to catch and process TransactionTimeoutException.

Integration With JTA

Ignite can be configured with a JTA transaction manager lookup class using TransactionConfiguration#setTxManagerFactory method. Transaction manager factory is a factory that provides Ignite with an instance of JTA transaction manager.

Ignite provides CacheJndiTmFactory factory. It's out-of-the-box transaction manager factory implementation that is using JNDI names to find TM.

When set, on each cache operation on a transactional cache Ignite will check if there is an ongoing JTA transaction. If JTA transaction is started, Ignite will also start a transaction and will enlist it into JTA transaction using it's own internal implementation of XAResource. Ignite transaction will be prepared, committed or rolledback altogether with corresponding JTA transaction.
Below is an example of using JTA transaction manager together with Ignite.

// Get an instance of JTA transaction manager.
TMService tms = appCtx.getComponent(TMService.class);

// Get an instance of Ignite cache.
IgniteCache<String, Integer> cache = cache();

UserTransaction jtaTx = tms.getUserTransaction();

// Start JTA transaction.
jtaTx.begin();

try {
    // Do some cache operations.
    cache.put("key1", 1);
    cache.put("key2", 2);

    // Commit the transaction.
    jtaTx.commit();
}
finally {
    // Rollback in a case of exception.
    if (jtaTx.getStatus() == Status.STATUS_ACTIVE)
        jtaTx.rollback();
}

Transactions

ACID compliant transactions ensuring guaranteed consistency.