This is an automated email from the ASF dual-hosted git repository.

av pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new 86bf2f03ef1 IGNITE-26435 Refactor IgniteTxEntry (#12800)
86bf2f03ef1 is described below

commit 86bf2f03ef128cda6d2f18c8b6664df1ba1c670a
Author: Anton Vinogradov <[email protected]>
AuthorDate: Fri Feb 27 16:29:51 2026 +0300

    IGNITE-26435 Refactor IgniteTxEntry (#12800)
---
 .../communication/GridIoMessageFactory.java        |   3 +-
 .../cache/transactions/IgniteTxEntry.java          | 287 ++-------------------
 .../cache/transactions/IgniteTxLocalAdapter.java   |   8 +-
 3 files changed, 30 insertions(+), 268 deletions(-)

diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 8756e664c1e..3396b8dad0b 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -246,6 +246,7 @@ import 
org.apache.ignite.internal.processors.cache.query.continuous.CacheContinu
 import 
org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryBatchAckSerializer;
 import 
org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
+import 
org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntrySerializer;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import 
org.apache.ignite.internal.processors.cache.transactions.IgniteTxKeySerializer;
 import 
org.apache.ignite.internal.processors.cache.transactions.TxEntryValueHolder;
@@ -454,7 +455,7 @@ public class GridIoMessageFactory implements 
MessageFactoryProvider {
         factory.register((short)96, CacheContinuousQueryEntry::new);
         factory.register((short)97, CacheEvictionEntry::new, new 
CacheEvictionEntrySerializer());
         factory.register((short)98, CacheEntryPredicateAdapter::new, new 
CacheEntryPredicateAdapterSerializer());
-        factory.register((short)100, IgniteTxEntry::new);
+        factory.register((short)100, IgniteTxEntry::new, new 
IgniteTxEntrySerializer());
         factory.register((short)101, TxEntryValueHolder::new, new 
TxEntryValueHolderSerializer());
         factory.register((short)102, CacheVersionedValue::new, new 
CacheVersionedValueSerializer());
         factory.register((short)103, GridCacheRawVersionedEntry::new);
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
index 3ac3db4b47f..1c80cfc75e0 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEntry.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
-import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.LinkedList;
 import java.util.UUID;
@@ -26,8 +25,7 @@ import javax.cache.expiry.ExpiryPolicy;
 import javax.cache.processor.EntryProcessor;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.GridDirectTransient;
-import org.apache.ignite.internal.IgniteCodeGeneratingFail;
+import org.apache.ignite.internal.Order;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
 import org.apache.ignite.internal.processors.cache.CacheInvalidStateException;
@@ -42,19 +40,14 @@ import 
org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import 
org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.lang.GridAbsClosureX;
 import org.apache.ignite.internal.util.lang.GridPeerDeployAware;
 import org.apache.ignite.internal.util.tostring.GridToStringBuilder;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.plugin.extensions.communication.Message;
-import 
org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
@@ -66,7 +59,6 @@ import static 
org.apache.ignite.internal.processors.cache.GridCacheOperation.TRA
  * {@link #equals(Object)} method, as transaction entries should use 
referential
  * equality.
  */
-@IgniteCodeGeneratingFail // Field filters should not be generated by 
MessageCodeGenerator.
 public class IgniteTxEntry implements GridPeerDeployAware, Message {
     /** */
     private static final long serialVersionUID = 0L;
@@ -107,125 +99,116 @@ public class IgniteTxEntry implements 
GridPeerDeployAware, Message {
 
     /** Owning transaction. */
     @GridToStringExclude
-    @GridDirectTransient
     public IgniteInternalTx tx;
 
     /** Cache key. */
     @GridToStringExclude
-    private KeyCacheObject key;
+    @Order(0)
+    KeyCacheObject key;
 
     /** Cache ID. */
     @GridToStringExclude
-    private int cacheId;
+    @Order(1)
+    int cacheId;
 
     /** Transient tx key. */
-    @GridDirectTransient
     private IgniteTxKey txKey;
 
     /** Cache value. */
     @GridToStringInclude
-    private TxEntryValueHolder val = new TxEntryValueHolder();
+    @Order(2)
+    TxEntryValueHolder val = new TxEntryValueHolder();
 
     /** Visible value for peek. */
     @GridToStringInclude
-    @GridDirectTransient
     private final TxEntryValueHolder prevVal = new TxEntryValueHolder();
 
     /** Old value before update. */
     @GridToStringInclude
-    private TxEntryValueHolder oldVal = new TxEntryValueHolder();
+    @Order(3)
+    TxEntryValueHolder oldVal = new TxEntryValueHolder();
 
     /** Transform. */
     @GridToStringInclude
-    @GridDirectTransient
     private Collection<T2<EntryProcessor<Object, Object, Object>, Object[]>> 
entryProcessorsCol;
 
     /** Transient field for calculated entry processor value. */
-    @GridDirectTransient
     private T2<GridCacheOperation, CacheObject> entryProcessorCalcVal;
 
     /** Transform closure bytes. */
     @GridToStringExclude
-    private byte[] transformClosBytes;
+    @Order(4)
+    byte[] transformClosBytes;
 
     /** Time to live. */
-    private long ttl;
+    @Order(5)
+    long ttl;
 
     /** DR expire time (explicit) */
-    private long conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
+    @Order(6)
+    long conflictExpireTime = CU.EXPIRE_TIME_CALCULATE;
 
     /** Conflict version. */
-    private GridCacheVersion conflictVer;
+    @Order(7)
+    GridCacheVersion conflictVer;
 
     /** Explicit lock version if there is one. */
     @GridToStringInclude
-    private GridCacheVersion explicitVer;
+    @Order(8)
+    GridCacheVersion explicitVer;
 
     /** DHT version. */
-    @GridDirectTransient
     private volatile GridCacheVersion dhtVer;
 
     /** Put filters. */
     @GridToStringInclude
-    private CacheEntryPredicate[] filters;
+    @Order(9)
+    CacheEntryPredicate[] filters;
 
     /** Flag indicating whether filters passed. Used for fast-commit 
transactions. */
-    @GridDirectTransient
     private boolean filtersPassed;
 
     /** Flag indicating that filter is set and can not be replaced. */
-    @GridDirectTransient
     private boolean filtersSet;
 
     /** Underlying cache entry. */
-    @GridDirectTransient
     private volatile GridCacheEntryEx entry;
 
     /** Cache registry. */
-    @GridDirectTransient
     private GridCacheContext<?, ?> ctx;
 
     /** Prepared flag to prevent multiple candidate add. */
-    @GridDirectTransient
     private transient volatile int prepared;
 
     /** Lock flag for collocated cache. */
-    @GridDirectTransient
     private transient boolean locked;
 
     /** Assigned node ID (required only for partitioned cache). */
-    @GridDirectTransient
     private UUID nodeId;
 
     /** Flag if this node is a back up node. */
-    @GridDirectTransient
     private boolean locMapped;
 
     /** Expiry policy. */
-    @GridDirectTransient
     private ExpiryPolicy expiryPlc;
 
     /** Expiry policy transfer flag. */
-    @GridDirectTransient
     private boolean transferExpiryPlc;
 
     /** Expiry policy bytes. */
-    private byte[] expiryPlcBytes;
+    @Order(10)
+    byte[] expiryPlcBytes;
 
     /** Additional flags. */
-    private byte flags;
+    @Order(11)
+    byte flags;
 
     /** Partition update counter. */
-    @GridDirectTransient
     private long partUpdateCntr;
 
     /** */
-    private GridCacheVersion serReadVer;
-
-    /** */
-    @GridDirectTransient
-    @GridToStringExclude
-    private transient @Nullable GridAbsClosureX cqNotifyC;
+    @Order(12)
+    GridCacheVersion serReadVer;
 
     /**
      * Empty constructor.
@@ -1109,216 +1092,6 @@ public class IgniteTxEntry implements 
GridPeerDeployAware, Message {
         serReadVer = null;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
-        switch (writer.state()) {
-            case 0:
-                if (!writer.writeInt(cacheId))
-                    return false;
-
-                writer.incrementState();
-
-            case 1:
-                if (!writer.writeLong(conflictExpireTime))
-                    return false;
-
-                writer.incrementState();
-
-            case 2:
-                if (!writer.writeMessage(conflictVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 3:
-                if (!writer.writeByteArray(expiryPlcBytes))
-                    return false;
-
-                writer.incrementState();
-
-            case 4:
-                if (!writer.writeMessage(explicitVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
-                if (!writer.writeObjectArray(
-                    !F.isEmptyOrNulls(filters) ? filters : null, 
MessageCollectionItemType.MSG))
-                    return false;
-
-                writer.incrementState();
-
-            case 6:
-                if (!writer.writeByte(flags))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
-                if (!writer.writeKeyCacheObject(key))
-                    return false;
-
-                writer.incrementState();
-
-            case 8:
-                if (!writer.writeMessage(oldVal))
-                    return false;
-
-                writer.incrementState();
-
-            case 9:
-                if (!writer.writeMessage(serReadVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 10:
-                if (!writer.writeByteArray(transformClosBytes))
-                    return false;
-
-                writer.incrementState();
-
-            case 11:
-                if (!writer.writeLong(ttl))
-                    return false;
-
-                writer.incrementState();
-
-            case 12:
-                if (!writer.writeMessage(val))
-                    return false;
-
-                writer.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        switch (reader.state()) {
-            case 0:
-                cacheId = reader.readInt();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 1:
-                conflictExpireTime = reader.readLong();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 2:
-                conflictVer = reader.readMessage();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 3:
-                expiryPlcBytes = reader.readByteArray();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 4:
-                explicitVer = reader.readMessage();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
-                filters = 
reader.readObjectArray(MessageCollectionItemType.MSG, 
CacheEntryPredicate.class);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 6:
-                flags = reader.readByte();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
-                key = reader.readKeyCacheObject();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 8:
-                oldVal = reader.readMessage();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 9:
-                serReadVer = reader.readMessage();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 10:
-                transformClosBytes = reader.readByteArray();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 11:
-                ttl = reader.readLong();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 12:
-                val = reader.readMessage();
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return true;
-    }
-
     /** {@inheritDoc} */
     @Override public short directType() {
         return 100;
@@ -1335,12 +1108,6 @@ public class IgniteTxEntry implements 
GridPeerDeployAware, Message {
             key.getClass() : val != null ? val.getClass() : getClass();
     }
 
-    /**
-     */
-    public GridAbsClosureX cqNotifyClosure() {
-        return cqNotifyC;
-    }
-
     /** {@inheritDoc} */
     @Override public ClassLoader classLoader() {
         return deployClass().getClassLoader();
diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index b027ff10719..fb9bc8b81ca 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -849,15 +849,9 @@ public abstract class IgniteTxLocalAdapter extends 
IgniteTxAdapter implements Ig
                     }
                 }
 
-                if (txCounters != null) {
+                if (txCounters != null)
                     
cctx.tm().txHandler().applyPartitionsUpdatesCounters(txCounters.updateCounters());
 
-                    for (IgniteTxEntry entry : commitEntries) {
-                        if (entry.cqNotifyClosure() != null)
-                            entry.cqNotifyClosure().applyx();
-                    }
-                }
-
                 // Apply cache sizes only for primary nodes. Update counters 
were applied on prepare state.
                 applyTxSizes();
 

Reply via email to