diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java index d55e205bdefff..5cb6a27b1f77d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobCancelRequest.java @@ -131,33 +131,33 @@ public boolean system() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("jobId", jobId)) + if (!writer.writeIgniteUuid("jobId", jobId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putBoolean("sys", sys)) + if (!writer.writeBoolean("sys", sys)) return false; - commState.idx++; + state++; } @@ -167,32 +167,32 @@ public boolean system() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - jobId = commState.getGridUuid("jobId"); + jobId = reader.readIgniteUuid("jobId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - sys = commState.getBoolean("sys"); + sys = reader.readBoolean("sys"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java index cdc6dfaa3c918..46310005f56ea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteRequest.java @@ -456,194 +456,141 @@ public UUID getSubjectId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putString("cpSpi", cpSpi)) + if (!writer.writeString("cpSpi", cpSpi)) return false; - commState.idx++; + state++; case 2: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 3: - if (!commState.putBoolean("dynamicSiblings", dynamicSiblings)) + if (!writer.writeBoolean("dynamicSiblings", dynamicSiblings)) return false; - commState.idx++; + state++; case 4: - if (!commState.putBoolean("forceLocDep", forceLocDep)) + if (!writer.writeBoolean("forceLocDep", forceLocDep)) return false; - commState.idx++; + state++; case 5: - if (!commState.putBoolean("internal", internal)) + if (!writer.writeBoolean("internal", internal)) return false; - commState.idx++; + state++; case 6: - if (!commState.putByteArray("jobAttrsBytes", jobAttrsBytes)) + if (!writer.writeByteArray("jobAttrsBytes", jobAttrsBytes)) return false; - commState.idx++; + state++; case 7: - if (!commState.putByteArray("jobBytes", jobBytes)) + if (!writer.writeByteArray("jobBytes", jobBytes)) return false; - commState.idx++; + state++; case 8: - if (!commState.putGridUuid("jobId", jobId)) + if (!writer.writeIgniteUuid("jobId", jobId)) return false; - commState.idx++; + state++; case 9: - if (ldrParticipants != null) { - if (commState.it == null) { - if (!commState.putInt(null, ldrParticipants.size())) - return false; - - commState.it = ldrParticipants.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("ldrParticipants", ldrParticipants, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 10: - if (!commState.putByteArray("sesAttrsBytes", sesAttrsBytes)) + if (!writer.writeByteArray("sesAttrsBytes", sesAttrsBytes)) return false; - commState.idx++; + state++; case 11: - if (!commState.putBoolean("sesFullSup", sesFullSup)) + if (!writer.writeBoolean("sesFullSup", sesFullSup)) return false; - commState.idx++; + state++; case 12: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; case 13: - if (!commState.putByteArray("siblingsBytes", siblingsBytes)) + if (!writer.writeByteArray("siblingsBytes", siblingsBytes)) return false; - commState.idx++; + state++; case 14: - if (!commState.putLong("startTaskTime", startTaskTime)) + if (!writer.writeLong("startTaskTime", startTaskTime)) return false; - commState.idx++; + state++; case 15: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 16: - if (!commState.putString("taskClsName", taskClsName)) + if (!writer.writeString("taskClsName", taskClsName)) return false; - commState.idx++; + state++; case 17: - if (!commState.putString("taskName", taskName)) + if (!writer.writeString("taskName", taskName)) return false; - commState.idx++; + state++; case 18: - if (!commState.putLong("timeout", timeout)) + if (!writer.writeLong("timeout", timeout)) return false; - commState.idx++; + state++; case 19: - if (top != null) { - if (commState.it == null) { - if (!commState.putInt(null, top.size())) - return false; - - commState.it = top.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putUuid(null, (UUID)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("top", top, UUID.class)) + return false; - commState.idx++; + state++; case 20: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -653,235 +600,176 @@ public UUID getSubjectId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - cpSpi = commState.getString("cpSpi"); + cpSpi = reader.readString("cpSpi"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 3: - dynamicSiblings = commState.getBoolean("dynamicSiblings"); + dynamicSiblings = reader.readBoolean("dynamicSiblings"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - forceLocDep = commState.getBoolean("forceLocDep"); + forceLocDep = reader.readBoolean("forceLocDep"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - internal = commState.getBoolean("internal"); + internal = reader.readBoolean("internal"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - jobAttrsBytes = commState.getByteArray("jobAttrsBytes"); + jobAttrsBytes = reader.readByteArray("jobAttrsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - jobBytes = commState.getByteArray("jobBytes"); + jobBytes = reader.readByteArray("jobBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - jobId = commState.getGridUuid("jobId"); + jobId = reader.readIgniteUuid("jobId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + ldrParticipants = reader.readMap("ldrParticipants", UUID.class, IgniteUuid.class); - if (commState.readSize >= 0) { - if (ldrParticipants == null) - ldrParticipants = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - ldrParticipants.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 10: - sesAttrsBytes = commState.getByteArray("sesAttrsBytes"); + sesAttrsBytes = reader.readByteArray("sesAttrsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - sesFullSup = commState.getBoolean("sesFullSup"); + sesFullSup = reader.readBoolean("sesFullSup"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - siblingsBytes = commState.getByteArray("siblingsBytes"); + siblingsBytes = reader.readByteArray("siblingsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - startTaskTime = commState.getLong("startTaskTime"); + startTaskTime = reader.readLong("startTaskTime"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 15: - subjId = commState.getUuid("subjId"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 16: - taskClsName = commState.getString("taskClsName"); + taskClsName = reader.readString("taskClsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - taskName = commState.getString("taskName"); + taskName = reader.readString("taskName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - timeout = commState.getLong("timeout"); + timeout = reader.readLong("timeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 19: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (top == null) - top = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - UUID _val = commState.getUuid(null); + top = reader.readCollection("top", UUID.class); - if (!commState.lastRead()) - return false; - - top.add((UUID)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 20: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java index 5d936c8f64461..3c3352a72b469 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobExecuteResponse.java @@ -234,57 +234,57 @@ public void setFakeException(IgniteException fakeEx) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("gridExBytes", gridExBytes)) + if (!writer.writeByteArray("gridExBytes", gridExBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("isCancelled", isCancelled)) + if (!writer.writeBoolean("isCancelled", isCancelled)) return false; - commState.idx++; + state++; case 2: - if (!commState.putByteArray("jobAttrsBytes", jobAttrsBytes)) + if (!writer.writeByteArray("jobAttrsBytes", jobAttrsBytes)) return false; - commState.idx++; + state++; case 3: - if (!commState.putGridUuid("jobId", jobId)) + if (!writer.writeIgniteUuid("jobId", jobId)) return false; - commState.idx++; + state++; case 4: - if (!commState.putUuid("nodeId", nodeId)) + if (!writer.writeUuid("nodeId", nodeId)) return false; - commState.idx++; + state++; case 5: - if (!commState.putByteArray("resBytes", resBytes)) + if (!writer.writeByteArray("resBytes", resBytes)) return false; - commState.idx++; + state++; case 6: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; } @@ -294,64 +294,64 @@ public void setFakeException(IgniteException fakeEx) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - gridExBytes = commState.getByteArray("gridExBytes"); + gridExBytes = reader.readByteArray("gridExBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - isCancelled = commState.getBoolean("isCancelled"); + isCancelled = reader.readBoolean("isCancelled"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - jobAttrsBytes = commState.getByteArray("jobAttrsBytes"); + jobAttrsBytes = reader.readByteArray("jobAttrsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - jobId = commState.getGridUuid("jobId"); + jobId = reader.readIgniteUuid("jobId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - nodeId = commState.getUuid("nodeId"); + nodeId = reader.readUuid("nodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - resBytes = commState.getByteArray("resBytes"); + resBytes = reader.readByteArray("resBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java index db27003c465bc..2fc71d9912c6c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsRequest.java @@ -105,27 +105,27 @@ public byte[] topicBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putByteArray("topicBytes", topicBytes)) + if (!writer.writeByteArray("topicBytes", topicBytes)) return false; - commState.idx++; + state++; } @@ -135,24 +135,24 @@ public byte[] topicBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - topicBytes = commState.getByteArray("topicBytes"); + topicBytes = reader.readByteArray("topicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java index 0dad8d914ab09..ffaaa1ef8cd10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingsResponse.java @@ -97,21 +97,21 @@ public void unmarshalSiblings(Marshaller marsh) throws IgniteCheckedException { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("siblingsBytes", siblingsBytes)) + if (!writer.writeByteArray("siblingsBytes", siblingsBytes)) return false; - commState.idx++; + state++; } @@ -121,16 +121,16 @@ public void unmarshalSiblings(Marshaller marsh) throws IgniteCheckedException { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - siblingsBytes = commState.getByteArray("siblingsBytes"); + siblingsBytes = reader.readByteArray("siblingsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java index 21153af7c1df4..75ff8eabec6a7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskCancelRequest.java @@ -80,21 +80,21 @@ public IgniteUuid sessionId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; } @@ -104,16 +104,16 @@ public IgniteUuid sessionId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java index f33e97ad966d6..1f75ee2bfe3af 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionRequest.java @@ -120,33 +120,33 @@ public IgniteUuid getJobId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("attrsBytes", attrsBytes)) + if (!writer.writeByteArray("attrsBytes", attrsBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("jobId", jobId)) + if (!writer.writeIgniteUuid("jobId", jobId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; } @@ -156,32 +156,32 @@ public IgniteUuid getJobId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - attrsBytes = commState.getByteArray("attrsBytes"); + attrsBytes = reader.readByteArray("attrsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - jobId = commState.getGridUuid("jobId"); + jobId = reader.readIgniteUuid("jobId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectByteBufferStream.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectByteBufferStream.java index 3234a64fc08b8..9d79f414e4e48 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectByteBufferStream.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectByteBufferStream.java @@ -969,7 +969,7 @@ public , T> C readCollection(Class itemCls) { } @SuppressWarnings("unchecked") - public Map readMap(Class keyCls, Class valCls) { + public , K, V> M readMap(Class keyCls, Class valCls) { if (readSize == -1) { int size = readInt(); @@ -1021,7 +1021,7 @@ public Map readMap(Class keyCls, Class valCls) { map = null; - return map0; + return (M)map0; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java index f0efbae1e9a27..1bc3d50f19c42 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/direct/DirectMessageReader.java @@ -263,8 +263,8 @@ public DirectMessageReader(GridTcpMessageFactory msgFactory) { } /** {@inheritDoc} */ - @Override public Map readMap(String name, Class keyCls, Class valCls) { - Map map = stream.readMap(keyCls, valCls); + @Override public , K, V> M readMap(String name, Class keyCls, Class valCls) { + M map = stream.readMap(keyCls, valCls); lastRead = stream.lastFinished(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/direct/GridTcpCommunicationMessageState.java b/modules/core/src/main/java/org/apache/ignite/internal/direct/GridTcpCommunicationMessageState.java deleted file mode 100644 index ad9542a6b5a1a..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/direct/GridTcpCommunicationMessageState.java +++ /dev/null @@ -1,726 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.direct; - -import org.apache.ignite.internal.processors.cache.*; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*; -import org.apache.ignite.internal.processors.cache.version.*; -import org.apache.ignite.internal.processors.clock.*; -import org.apache.ignite.internal.util.*; -import org.apache.ignite.lang.*; -import org.apache.ignite.plugin.extensions.communication.*; -import org.jetbrains.annotations.*; -import sun.misc.*; - -import java.nio.*; -import java.util.*; - -import static org.apache.ignite.plugin.extensions.communication.MessageAdapter.*; - -/** - * Communication message state. - */ -@SuppressWarnings("PublicField") -public class GridTcpCommunicationMessageState { - /** */ - private static final Unsafe UNSAFE = GridUnsafe.unsafe(); - - /** */ - private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class); - - /** */ - private final MessageWriter writer = new DirectMessageWriter(); - - /** */ - private MessageReader reader; - - /** */ - public int idx; - - /** */ - public boolean typeWritten; - - /** */ - public Iterator it; - - /** */ - public Object cur; - - /** */ - public boolean keyDone; - - /** */ - public int readSize = -1; - - /** */ - public int readItems; - - /** - * @param writer Writer. - */ - public final void setWriter(MessageWriter writer) { -// if (this.writer == null) -// this.writer = writer; - } - - /** - * @param reader Reader. - */ - public final void setReader(MessageReader reader) { - if (this.reader == null) - this.reader = reader; - } - - /** - * @param buf Buffer. - */ - public final void setBuffer(ByteBuffer buf) { - if (writer != null) - writer.setBuffer(buf); - - if (reader != null) - reader.setBuffer(buf); - } - - /** - * @param name Field name. - * @param b Byte value. - * @return Whether value was written. - */ - public final boolean putByte(String name, byte b) { - return writer.writeByte(name, b); - } - - /** - * @param name Field name. - * @return Byte value. - */ - public final byte getByte(String name) { - return reader.readByte(name); - } - - /** - * @param name Field name. - * @param s Short value. - * @return Whether value was written. - */ - public final boolean putShort(String name, short s) { - return writer.writeShort(name, s); - } - - /** - * @param name Field name. - * @return Short value. - */ - public final short getShort(String name) { - return reader.readShort(name); - } - - /** - * @param name Field name. - * @param i Integer value. - * @return Whether value was written. - */ - public final boolean putInt(String name, int i) { - return writer.writeInt(name, i); - } - - /** - * @param name Field name. - * @return Integer value. - */ - public final int getInt(String name) { - return reader.readInt(name); - } - - /** - * @param name Field name. - * @param l Long value. - * @return Whether value was written. - */ - public final boolean putLong(String name, long l) { - return writer.writeLong(name, l); - } - - /** - * @param name Field name. - * @return Long value. - */ - public final long getLong(String name) { - return reader.readLong(name); - } - - /** - * @param name Field name. - * @param f Float value. - * @return Whether value was written. - */ - public final boolean putFloat(String name, float f) { - return writer.writeFloat(name, f); - } - - /** - * @param name Field name. - * @return Float value. - */ - public final float getFloat(String name) { - return reader.readFloat(name); - } - - /** - * @param name Field name. - * @param d Double value. - * @return Whether value was written. - */ - public final boolean putDouble(String name, double d) { - return writer.writeDouble(name, d); - } - - /** - * @param name Field name. - * @return Double value. - */ - public final double getDouble(String name) { - return reader.readDouble(name); - } - - /** - * @param name Field name. - * @param c Char value. - * @return Whether value was written. - */ - public final boolean putChar(String name, char c) { - return writer.writeChar(name, c); - } - - /** - * @param name Field name. - * @return Char value. - */ - public final char getChar(String name) { - return reader.readChar(name); - } - - /** - * @param name Field name. - * @param b Boolean value. - * @return Whether value was written. - */ - public final boolean putBoolean(String name, boolean b) { - return writer.writeBoolean(name, b); - } - - /** - * @param name Field name. - * @return Boolean value. - */ - public final boolean getBoolean(String name) { - return reader.readBoolean(name); - } - - /** - * @param name Field name. - * @param arr Byte array. - * @return Whether array was fully written. - */ - public final boolean putByteArray(String name, @Nullable byte[] arr) { - return writer.writeByteArray(name, arr); - } - - /** - * @param name Field name. - * @return Byte array. - */ - public final byte[] getByteArray(String name) { - return reader.readByteArray(name); - } - - /** - * @param name Field name. - * @param arr Short array. - * @return Whether array was fully written. - */ - public final boolean putShortArray(String name, short[] arr) { - return writer.writeShortArray(name, arr); - } - - /** - * @param name Field name. - * @return Short array. - */ - public final short[] getShortArray(String name) { - return reader.readShortArray(name); - } - - /** - * @param name Field name. - * @param arr Integer array. - * @return Whether array was fully written. - */ - public final boolean putIntArray(String name, int[] arr) { - return writer.writeIntArray(name, arr); - } - - /** - * @param name Field name. - * @return Integer array. - */ - public final int[] getIntArray(String name) { - return reader.readIntArray(name); - } - - /** - * @param name Field name. - * @param arr Long array. - * @return Whether array was fully written. - */ - public final boolean putLongArray(String name, long[] arr) { - return writer.writeLongArray(name, arr); - } - - /** - * @param name Field name. - * @return Long array. - */ - public final long[] getLongArray(String name) { - return reader.readLongArray(name); - } - - /** - * @param name Field name. - * @param arr Float array. - * @return Whether array was fully written. - */ - public final boolean putFloatArray(String name, float[] arr) { - return writer.writeFloatArray(name, arr); - } - - /** - * @param name Field name. - * @return Float array. - */ - public final float[] getFloatArray(String name) { - return reader.readFloatArray(name); - } - - /** - * @param name Field name. - * @param arr Double array. - * @return Whether array was fully written. - */ - public final boolean putDoubleArray(String name, double[] arr) { - return writer.writeDoubleArray(name, arr); - } - - /** - * @param name Field name. - * @return Double array. - */ - public final double[] getDoubleArray(String name) { - return reader.readDoubleArray(name); - } - - /** - * @param name Field name. - * @param arr Char array. - * @return Whether array was fully written. - */ - public final boolean putCharArray(String name, char[] arr) { - return writer.writeCharArray(name, arr); - } - - /** - * @param name Field name. - * @return Char array. - */ - public final char[] getCharArray(String name) { - return reader.readCharArray(name); - } - - /** - * @param name Field name. - * @param arr Boolean array. - * @return Whether array was fully written. - */ - public final boolean putBooleanArray(String name, boolean[] arr) { - return writer.writeBooleanArray(name, arr); - } - - /** - * @param name Field name. - * @return Boolean array. - */ - public final boolean[] getBooleanArray(String name) { - return reader.readBooleanArray(name); - } - - - /** - * @param name Field name. - * @param uuid {@link UUID}. - * @return Whether value was fully written. - */ - public final boolean putUuid(String name, @Nullable UUID uuid) { - byte[] arr = null; - - if (uuid != null) { - arr = new byte[16]; - - UNSAFE.putLong(arr, BYTE_ARR_OFF, uuid.getMostSignificantBits()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 8, uuid.getLeastSignificantBits()); - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link UUID}. - */ - public final UUID getUuid(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - long most = UNSAFE.getLong(arr, BYTE_ARR_OFF); - long least = UNSAFE.getLong(arr, BYTE_ARR_OFF + 8); - - return new UUID(most, least); - } - } - - /** - * @param name Field name. - * @param uuid {@link IgniteUuid}. - * @return Whether value was fully written. - */ - public final boolean putGridUuid(String name, @Nullable IgniteUuid uuid) { - byte[] arr = null; - - if (uuid != null) { - arr = new byte[24]; - - UNSAFE.putLong(arr, BYTE_ARR_OFF, uuid.globalId().getMostSignificantBits()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 8, uuid.globalId().getLeastSignificantBits()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 16, uuid.localId()); - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link IgniteUuid}. - */ - public final IgniteUuid getGridUuid(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - long most = UNSAFE.getLong(arr, BYTE_ARR_OFF); - long least = UNSAFE.getLong(arr, BYTE_ARR_OFF + 8); - long loc = UNSAFE.getLong(arr, BYTE_ARR_OFF + 16); - - return new IgniteUuid(new UUID(most, least), loc); - } - } - - /** - * @param name Field name. - * @param ver {@link GridClockDeltaVersion}. - * @return Whether value was fully written. - */ - public final boolean putClockDeltaVersion(String name, @Nullable GridClockDeltaVersion ver) { - byte[] arr = null; - - if (ver != null) { - arr = new byte[16]; - - UNSAFE.putLong(arr, BYTE_ARR_OFF, ver.version()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 8, ver.topologyVersion()); - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link GridClockDeltaVersion}. - */ - public final GridClockDeltaVersion getClockDeltaVersion(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - long ver = UNSAFE.getLong(arr, BYTE_ARR_OFF); - long topVer = UNSAFE.getLong(arr, BYTE_ARR_OFF + 8); - - return new GridClockDeltaVersion(ver, topVer); - } - } - - /** - * @param name Field name. - * @param list {@link GridByteArrayList}. - * @return Whether value was fully written. - */ - public final boolean putByteArrayList(String name, @Nullable GridByteArrayList list) { - return putByteArray(name, list != null ? list.array() : null); - } - - /** - * @param name Field name. - * @return {@link GridByteArrayList}. - */ - public final GridByteArrayList getByteArrayList(String name) { - byte[] arr = getByteArray(name); - - return arr != null ? new GridByteArrayList(arr) : null; - } - - /** - * @param name Field name. - * @param list {@link GridLongList}. - * @return Whether value was fully written. - */ - public final boolean putLongList(String name, @Nullable GridLongList list) { - return putLongArray(name, list != null ? list.array() : null); - } - - /** - * @param name Field name. - * @return {@link GridLongList}. - */ - public final GridLongList getLongList(String name) { - long[] arr = getLongArray(name); - - return arr != null ? new GridLongList(arr) : null; - } - - /** - * @param name Field name. - * @param ver {@link org.apache.ignite.internal.processors.cache.version.GridCacheVersion}. - * @return Whether value was fully written. - */ - public final boolean putCacheVersion(String name, @Nullable GridCacheVersion ver) { - byte[] arr = null; - - if (ver != null) { - arr = new byte[24]; - - UNSAFE.putInt(arr, BYTE_ARR_OFF, ver.topologyVersion()); - UNSAFE.putInt(arr, BYTE_ARR_OFF + 4, ver.nodeOrderAndDrIdRaw()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 8, ver.globalTime()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 16, ver.order()); - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link GridCacheVersion}. - */ - public final GridCacheVersion getCacheVersion(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - int topVerDrId = UNSAFE.getInt(arr, BYTE_ARR_OFF); - int nodeOrder = UNSAFE.getInt(arr, BYTE_ARR_OFF + 4); - long globalTime = UNSAFE.getLong(arr, BYTE_ARR_OFF + 8); - long order = UNSAFE.getLong(arr, BYTE_ARR_OFF + 16); - - return new GridCacheVersion(topVerDrId, nodeOrder, globalTime, order); - } - } - - /** - * @param name Field name. - * @param id {@link GridDhtPartitionExchangeId}. - * @return Whether value was fully written. - */ - public final boolean putDhtPartitionExchangeId(String name, @Nullable GridDhtPartitionExchangeId id) { - byte[] arr = null; - - if (id != null) { - arr = new byte[28]; - - UNSAFE.putLong(arr, BYTE_ARR_OFF, id.nodeId().getMostSignificantBits()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 8, id.nodeId().getLeastSignificantBits()); - UNSAFE.putInt(arr, BYTE_ARR_OFF + 16, id.event()); - UNSAFE.putLong(arr, BYTE_ARR_OFF + 20, id.topologyVersion()); - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link GridDhtPartitionExchangeId}. - */ - public final GridDhtPartitionExchangeId getDhtPartitionExchangeId(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - long most = UNSAFE.getLong(arr, BYTE_ARR_OFF); - long least = UNSAFE.getLong(arr, BYTE_ARR_OFF + 8); - int evt = UNSAFE.getInt(arr, BYTE_ARR_OFF + 16); - long topVer = UNSAFE.getLong(arr, BYTE_ARR_OFF + 20); - - return new GridDhtPartitionExchangeId(new UUID(most, least), evt, topVer); - } - } - - /** - * @param name Field name. - * @param bytes {@link GridCacheValueBytes}. - * @return Whether value was fully written. - */ - public final boolean putValueBytes(String name, @Nullable GridCacheValueBytes bytes) { - byte[] arr = null; - - if (bytes != null) { - byte[] bytes0 = bytes.get(); - - if (bytes0 != null) { - int len = bytes0.length; - - arr = new byte[len + 2]; - - UNSAFE.putBoolean(arr, BYTE_ARR_OFF, true); - UNSAFE.copyMemory(bytes0, BYTE_ARR_OFF, arr, BYTE_ARR_OFF + 1, len); - UNSAFE.putBoolean(arr, BYTE_ARR_OFF + 1 + len, bytes.isPlain()); - } - else { - arr = new byte[1]; - - UNSAFE.putBoolean(arr, BYTE_ARR_OFF, false); - } - } - - return putByteArray(name, arr); - } - - /** - * @param name Field name. - * @return {@link GridCacheValueBytes}. - */ - public final GridCacheValueBytes getValueBytes(String name) { - byte[] arr = getByteArray(name); - - if (arr == null) - return null; - else { - boolean notNull = UNSAFE.getBoolean(arr, BYTE_ARR_OFF); - - if (notNull) { - int len = arr.length - 2; - - assert len >= 0 : len; - - byte[] bytesArr = new byte[len]; - - UNSAFE.copyMemory(arr, BYTE_ARR_OFF + 1, bytesArr, BYTE_ARR_OFF, len); - - boolean isPlain = UNSAFE.getBoolean(arr, BYTE_ARR_OFF + 1 + len); - - return new GridCacheValueBytes(bytesArr, isPlain); - } - else - return new GridCacheValueBytes(); - } - } - - /** - * @param name Field name. - * @param str {@link String}. - * @return Whether value was fully written. - */ - public final boolean putString(String name, @Nullable String str) { - return putByteArray(name, str != null ? str.getBytes() : null); - } - - /** - * @param name Field name. - * @return {@link String}. - */ - public final String getString(String name) { - byte[] arr = getByteArray(name); - - return arr != null ? new String(arr) : null; - } - - /** - * @param name Field name. - * @param bits {@link BitSet}. - * @return Whether value was fully written. - */ - public final boolean putBitSet(String name, @Nullable BitSet bits) { - return putLongArray(name, bits != null ? bits.toLongArray() : null); - } - - /** - * @param name Field name. - * @return {@link BitSet}. - */ - public final BitSet getBitSet(String name) { - long[] arr = getLongArray(name); - - return arr != null ? BitSet.valueOf(arr) : null; - } - - /** - * @param name Field name. - * @param e Enum. - * @return Whether value was fully written. - */ - public final boolean putEnum(String name, @Nullable Enum e) { - return putByte(name, e != null ? (byte)e.ordinal() : -1); - } - - /** - * @param name Field name. - * @param msg {@link MessageAdapter}. - * @return Whether value was fully written. - */ - public final boolean putMessage(String name, @Nullable MessageAdapter msg) { - if (msg != null) - msg.setWriter(writer); - - return writer.writeMessage(name, msg); - } - - /** - * @param name Field name. - * @return {@link MessageAdapter}. - */ - public final MessageAdapter getMessage(String name) { - return reader.readMessage(name); - } - - public final boolean lastRead() { - return reader.isLastRead(); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java index fda129a4d8f6d..5085a9cdae501 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointRequest.java @@ -105,33 +105,33 @@ public String getCheckpointSpi() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putString("cpSpi", cpSpi)) + if (!writer.writeString("cpSpi", cpSpi)) return false; - commState.idx++; + state++; case 1: - if (!commState.putString("key", key)) + if (!writer.writeString("key", key)) return false; - commState.idx++; + state++; case 2: - if (!commState.putGridUuid("sesId", sesId)) + if (!writer.writeIgniteUuid("sesId", sesId)) return false; - commState.idx++; + state++; } @@ -141,32 +141,32 @@ public String getCheckpointSpi() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - cpSpi = commState.getString("cpSpi"); + cpSpi = reader.readString("cpSpi"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - key = commState.getString("key"); + key = reader.readString("key"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - sesId = commState.getGridUuid("sesId"); + sesId = reader.readIgniteUuid("sesId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java index 5dc52d86ebb23..27d64be594c7b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java @@ -200,63 +200,63 @@ boolean isOrdered() { _clone.ordered = ordered; _clone.timeout = timeout; _clone.skipOnTimeout = skipOnTimeout; - _clone.msg = msg != null ? (MessageAdapter)msg.clone() : null; + _clone.msg = msg; } /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putMessage("msg", msg)) + if (!writer.writeMessage("msg", msg != null ? msg.clone() : null)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("ordered", ordered)) + if (!writer.writeBoolean("ordered", ordered)) return false; - commState.idx++; + state++; case 2: - if (!commState.putEnum("plc", plc)) + if (!writer.writeEnum("plc", plc)) return false; - commState.idx++; + state++; case 3: - if (!commState.putBoolean("skipOnTimeout", skipOnTimeout)) + if (!writer.writeBoolean("skipOnTimeout", skipOnTimeout)) return false; - commState.idx++; + state++; case 4: - if (!commState.putLong("timeout", timeout)) + if (!writer.writeLong("timeout", timeout)) return false; - commState.idx++; + state++; case 5: - if (!commState.putByteArray("topicBytes", topicBytes)) + if (!writer.writeByteArray("topicBytes", topicBytes)) return false; - commState.idx++; + state++; case 6: - if (!commState.putInt("topicOrd", topicOrd)) + if (!writer.writeInt("topicOrd", topicOrd)) return false; - commState.idx++; + state++; } @@ -266,66 +266,64 @@ boolean isOrdered() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - msg = commState.getMessage("msg"); + msg = reader.readMessage("msg"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - ordered = commState.getBoolean("ordered"); + ordered = reader.readBoolean("ordered"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - byte plc0 = commState.getByte("plc"); + plc = reader.readEnum("plc", GridIoPolicy.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - plc = GridIoPolicy.fromOrdinal(plc0); - - commState.idx++; + state++; case 3: - skipOnTimeout = commState.getBoolean("skipOnTimeout"); + skipOnTimeout = reader.readBoolean("skipOnTimeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - timeout = commState.getLong("timeout"); + timeout = reader.readLong("timeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - topicBytes = commState.getByteArray("topicBytes"); + topicBytes = reader.readByteArray("topicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - topicOrd = commState.getInt("topicOrd"); + topicOrd = reader.readInt("topicOrd"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java index 44039a32d1169..0bc81d4e21bf6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoUserMessage.java @@ -231,89 +231,57 @@ public void deployment(GridDeployment dep) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("bodyBytes", bodyBytes)) + if (!writer.writeByteArray("bodyBytes", bodyBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putString("depClsName", depClsName)) + if (!writer.writeString("depClsName", depClsName)) return false; - commState.idx++; + state++; case 3: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 4: - if (ldrParties != null) { - if (commState.it == null) { - if (!commState.putInt(null, ldrParties.size())) - return false; - - commState.it = ldrParties.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("ldrParties", ldrParties, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 5: - if (!commState.putByteArray("topicBytes", topicBytes)) + if (!writer.writeByteArray("topicBytes", topicBytes)) return false; - commState.idx++; + state++; case 6: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -323,101 +291,64 @@ public void deployment(GridDeployment dep) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - bodyBytes = commState.getByteArray("bodyBytes"); + bodyBytes = reader.readByteArray("bodyBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - depClsName = commState.getString("depClsName"); + depClsName = reader.readString("depClsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + ldrParties = reader.readMap("ldrParties", UUID.class, IgniteUuid.class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (ldrParties == null) - ldrParties = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - ldrParties.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - topicBytes = commState.getByteArray("topicBytes"); + topicBytes = reader.readByteArray("topicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java index c0698d06f91a2..4907c56afc446 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentInfoBean.java @@ -161,77 +161,45 @@ public void localDeploymentOwner(boolean locDepOwner) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 2: - if (!commState.putBoolean("locDepOwner", locDepOwner)) + if (!writer.writeBoolean("locDepOwner", locDepOwner)) return false; - commState.idx++; + state++; case 3: - if (participants != null) { - if (commState.it == null) { - if (!commState.putInt(null, participants.size())) - return false; - - commState.it = participants.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("participants", participants, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 4: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -241,85 +209,48 @@ public void localDeploymentOwner(boolean locDepOwner) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 2: - locDepOwner = commState.getBoolean("locDepOwner"); + locDepOwner = reader.readBoolean("locDepOwner"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + participants = reader.readMap("participants", UUID.class, IgniteUuid.class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (participants == null) - participants = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - participants.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 4: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java index 86019b9a6e132..50d9b24fd62ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentRequest.java @@ -180,66 +180,45 @@ public void nodeIds(Collection nodeIds) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putBoolean("isUndeploy", isUndeploy)) + if (!writer.writeBoolean("isUndeploy", isUndeploy)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("ldrId", ldrId)) + if (!writer.writeIgniteUuid("ldrId", ldrId)) return false; - commState.idx++; + state++; case 2: - if (nodeIds != null) { - if (commState.it == null) { - if (!commState.putInt(null, nodeIds.size())) - return false; - - commState.it = nodeIds.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putUuid(null, (UUID)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nodeIds", nodeIds, UUID.class)) + return false; - commState.idx++; + state++; case 3: - if (!commState.putByteArray("resTopicBytes", resTopicBytes)) + if (!writer.writeByteArray("resTopicBytes", resTopicBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putString("rsrcName", rsrcName)) + if (!writer.writeString("rsrcName", rsrcName)) return false; - commState.idx++; + state++; } @@ -249,70 +228,48 @@ public void nodeIds(Collection nodeIds) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - isUndeploy = commState.getBoolean("isUndeploy"); + isUndeploy = reader.readBoolean("isUndeploy"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - ldrId = commState.getGridUuid("ldrId"); + ldrId = reader.readIgniteUuid("ldrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nodeIds = reader.readCollection("nodeIds", UUID.class); - if (commState.readSize >= 0) { - if (nodeIds == null) - nodeIds = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - nodeIds.add((UUID)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 3: - resTopicBytes = commState.getByteArray("resTopicBytes"); + resTopicBytes = reader.readByteArray("resTopicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - rsrcName = commState.getString("rsrcName"); + rsrcName = reader.readString("rsrcName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java index de50cc973e305..c46d7f332d8a9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentResponse.java @@ -127,33 +127,33 @@ void errorMessage(String errMsg) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArrayList("byteSrc", byteSrc)) + if (!writer.writeMessage("byteSrc", byteSrc != null ? byteSrc.clone() : null)) return false; - commState.idx++; + state++; case 1: - if (!commState.putString("errMsg", errMsg)) + if (!writer.writeString("errMsg", errMsg)) return false; - commState.idx++; + state++; case 2: - if (!commState.putBoolean("success", success)) + if (!writer.writeBoolean("success", success)) return false; - commState.idx++; + state++; } @@ -163,32 +163,32 @@ void errorMessage(String errMsg) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - byteSrc = commState.getByteArrayList("byteSrc"); + byteSrc = reader.readMessage("byteSrc"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - errMsg = commState.getString("errMsg"); + errMsg = reader.readString("errMsg"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - success = commState.getBoolean("success"); + success = reader.readBoolean("success"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java index 1658b88337bc7..352d3603edc39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageMessage.java @@ -291,101 +291,69 @@ void exceptionBytes(byte[] exBytes) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 2: - if (!commState.putByteArray("evtsBytes", evtsBytes)) + if (!writer.writeByteArray("evtsBytes", evtsBytes)) return false; - commState.idx++; + state++; case 3: - if (!commState.putByteArray("exBytes", exBytes)) + if (!writer.writeByteArray("exBytes", exBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("filter", filter)) + if (!writer.writeByteArray("filter", filter)) return false; - commState.idx++; + state++; case 5: - if (!commState.putString("filterClsName", filterClsName)) + if (!writer.writeString("filterClsName", filterClsName)) return false; - commState.idx++; + state++; case 6: - if (ldrParties != null) { - if (commState.it == null) { - if (!commState.putInt(null, ldrParties.size())) - return false; - - commState.it = ldrParties.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("ldrParties", ldrParties, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putByteArray("resTopicBytes", resTopicBytes)) + if (!writer.writeByteArray("resTopicBytes", resTopicBytes)) return false; - commState.idx++; + state++; case 8: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -395,117 +363,80 @@ void exceptionBytes(byte[] exBytes) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 2: - evtsBytes = commState.getByteArray("evtsBytes"); + evtsBytes = reader.readByteArray("evtsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - exBytes = commState.getByteArray("exBytes"); + exBytes = reader.readByteArray("exBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - filter = commState.getByteArray("filter"); + filter = reader.readByteArray("filter"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - filterClsName = commState.getString("filterClsName"); + filterClsName = reader.readString("filterClsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + ldrParties = reader.readMap("ldrParties", UUID.class, IgniteUuid.class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (ldrParties == null) - ldrParties = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - ldrParties.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - resTopicBytes = commState.getByteArray("resTopicBytes"); + resTopicBytes = reader.readByteArray("resTopicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java index bb63b86e20a33..ce7d4d763a36b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionRequest.java @@ -164,36 +164,36 @@ void addKey(K key, GridCacheVersion ver, boolean near) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("entriesBytes", entriesBytes)) + if (!writer.writeByteArray("entriesBytes", entriesBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putLong("futId", futId)) + if (!writer.writeLong("futId", futId)) return false; - commState.idx++; + state++; case 5: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -203,35 +203,35 @@ void addKey(K key, GridCacheVersion ver, boolean near) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - entriesBytes = commState.getByteArray("entriesBytes"); + entriesBytes = reader.readByteArray("entriesBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - futId = commState.getLong("futId"); + futId = reader.readLong("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionResponse.java index 819902058e011..2687e95b07318 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionResponse.java @@ -153,57 +153,36 @@ boolean error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putBoolean("err", err)) + if (!writer.writeBoolean("err", err)) return false; - commState.idx++; + state++; case 4: - if (!commState.putLong("futId", futId)) + if (!writer.writeLong("futId", futId)) return false; - commState.idx++; + state++; case 5: - if (rejectedKeyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, rejectedKeyBytes.size())) - return false; - - commState.it = rejectedKeyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("rejectedKeyBytes", rejectedKeyBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -213,57 +192,35 @@ boolean error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - err = commState.getBoolean("err"); + err = reader.readBoolean("err"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - futId = commState.getLong("futId"); + futId = reader.readLong("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + rejectedKeyBytes = reader.readCollection("rejectedKeyBytes", byte[].class); - if (commState.readSize >= 0) { - if (rejectedKeyBytes == null) - rejectedKeyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - rejectedKeyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java index 78028b0d5c2a3..dc8feee5dbb3c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java @@ -648,7 +648,7 @@ protected final void unmarshalTx(Iterable> txEntries, GridCacheMessage _clone = (GridCacheMessage)_msg; _clone.msgId = msgId; - _clone.depInfo = depInfo != null ? (GridDeploymentInfoBean)depInfo.clone() : null; + _clone.depInfo = depInfo; _clone.err = err; _clone.skipPrepare = skipPrepare; _clone.cacheId = cacheId; @@ -657,33 +657,33 @@ protected final void unmarshalTx(Iterable> txEntries, /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putInt("cacheId", cacheId)) + if (!writer.writeInt("cacheId", cacheId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putMessage("depInfo", depInfo)) + if (!writer.writeMessage("depInfo", depInfo != null ? depInfo.clone() : null)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong("msgId", msgId)) + if (!writer.writeLong("msgId", msgId)) return false; - commState.idx++; + state++; } @@ -693,32 +693,32 @@ protected final void unmarshalTx(Iterable> txEntries, /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - cacheId = commState.getInt("cacheId"); + cacheId = reader.readInt("cacheId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - depInfo = (GridDeploymentInfoBean)commState.getMessage("depInfo"); + depInfo = reader.readMessage("depInfo"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - msgId = commState.getLong("msgId"); + msgId = reader.readLong("msgId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueBytes.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueBytes.java index aba1c18d59fd0..289db2e08dd5f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueBytes.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheValueBytes.java @@ -118,12 +118,57 @@ public boolean isNull() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeByteArray("bytes", bytes)) + return false; + + state++; + + case 1: + if (!writer.writeBoolean("plain", plain)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + bytes = reader.readByteArray("bytes"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + plain = reader.readBoolean("plain"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -133,12 +178,19 @@ public boolean isNull() { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridCacheValueBytes _clone = new GridCacheValueBytes(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridCacheValueBytes _clone = (GridCacheValueBytes)_msg; + + _clone.bytes = bytes; + _clone.plain = plain; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java index e021408985679..914f15098becc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxRequest.java @@ -121,42 +121,42 @@ public int transactions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 10: - if (!commState.putCacheVersion("nearXidVer", nearXidVer)) + if (!writer.writeMessage("nearXidVer", nearXidVer != null ? nearXidVer.clone() : null)) return false; - commState.idx++; + state++; case 11: - if (!commState.putInt("txNum", txNum)) + if (!writer.writeInt("txNum", txNum)) return false; - commState.idx++; + state++; } @@ -166,43 +166,43 @@ public int transactions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - nearXidVer = commState.getCacheVersion("nearXidVer"); + nearXidVer = reader.readMessage("nearXidVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - txNum = commState.getInt("txNum"); + txNum = reader.readInt("txNum"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxResponse.java index bba1efb84ce48..7c9ae63e3273f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxResponse.java @@ -108,36 +108,36 @@ public boolean success() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 10: - if (!commState.putBoolean("success", success)) + if (!writer.writeBoolean("success", success)) return false; - commState.idx++; + state++; } @@ -147,35 +147,35 @@ public boolean success() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - success = commState.getBoolean("success"); + success = reader.readBoolean("success"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java index bd5f826c9b012..0aeabaf2075a2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxRequest.java @@ -154,54 +154,54 @@ public boolean nearOnlyCheck() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 10: - if (!commState.putCacheVersion("nearXidVer", nearXidVer)) + if (!writer.writeBoolean("nearOnlyCheck", nearOnlyCheck)) return false; - commState.idx++; + state++; case 11: - if (!commState.putUuid("originatingNodeId", originatingNodeId)) + if (!writer.writeMessage("nearXidVer", nearXidVer != null ? nearXidVer.clone() : null)) return false; - commState.idx++; + state++; case 12: - if (!commState.putLong("originatingThreadId", originatingThreadId)) + if (!writer.writeUuid("originatingNodeId", originatingNodeId)) return false; - commState.idx++; + state++; case 13: - if (!commState.putBoolean("nearOnlyCheck", nearOnlyCheck)) + if (!writer.writeLong("originatingThreadId", originatingThreadId)) return false; - commState.idx++; + state++; } @@ -211,59 +211,59 @@ public boolean nearOnlyCheck() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - nearXidVer = commState.getCacheVersion("nearXidVer"); + nearOnlyCheck = reader.readBoolean("nearOnlyCheck"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - originatingNodeId = commState.getUuid("originatingNodeId"); + nearXidVer = reader.readMessage("nearXidVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - originatingThreadId = commState.getLong("originatingThreadId"); + originatingNodeId = reader.readUuid("originatingNodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - nearOnlyCheck = commState.getBoolean("nearOnlyCheck"); + originatingThreadId = reader.readLong("originatingThreadId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxResponse.java index 68495f3003a51..36983beb66ff6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxResponse.java @@ -140,36 +140,36 @@ public GridCacheCommittedTxInfo committedTxInfo() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putByteArray("committedTxInfoBytes", committedTxInfoBytes)) + if (!writer.writeByteArray("committedTxInfoBytes", committedTxInfoBytes)) return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 10: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; } @@ -179,35 +179,35 @@ public GridCacheCommittedTxInfo committedTxInfo() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - committedTxInfoBytes = commState.getByteArray("committedTxInfoBytes"); + committedTxInfoBytes = reader.readByteArray("committedTxInfoBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java index f8f9870dca9ab..96eb0a467a494 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTtlUpdateRequest.java @@ -195,138 +195,54 @@ public List nearVersions() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (keysBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keysBytes.size())) - return false; - - commState.it = keysBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("keysBytes", keysBytes, byte[].class)) + return false; - commState.idx++; + state++; case 4: - if (nearKeysBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearKeysBytes.size())) - return false; - - commState.it = nearKeysBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearKeysBytes", nearKeysBytes, byte[].class)) + return false; - commState.idx++; + state++; case 5: - if (nearVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearVers.size())) - return false; - - commState.it = nearVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearVers", nearVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 6: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("ttl", ttl)) + if (!writer.writeLong("ttl", ttl)) return false; - commState.idx++; + state++; case 8: - if (vers != null) { - if (commState.it == null) { - if (!commState.putInt(null, vers.size())) - return false; - - commState.it = vers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("vers", vers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; } @@ -335,147 +251,59 @@ public List nearVersions() { /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (keysBytes == null) - keysBytes = new ArrayList<>(commState.readSize); + keysBytes = reader.readCollection("keysBytes", byte[].class); - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - keysBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearKeysBytes == null) - nearKeysBytes = new ArrayList<>(commState.readSize); + nearKeysBytes = reader.readCollection("nearKeysBytes", byte[].class); - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearKeysBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nearVers = reader.readCollection("nearVers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (nearVers == null) - nearVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - nearVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 6: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - ttl = commState.getLong("ttl"); + ttl = reader.readLong("ttl"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + vers = reader.readCollection("vers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (vers == null) - vers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - vers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java index 55e5250c245a4..0c82e020609d3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedBaseMessage.java @@ -258,90 +258,48 @@ public int keysCount() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("candsByIdxBytes", candsByIdxBytes)) + if (!writer.writeByteArray("candsByIdxBytes", candsByIdxBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("candsByKeyBytes", candsByKeyBytes)) + if (!writer.writeByteArray("candsByKeyBytes", candsByKeyBytes)) return false; - commState.idx++; + state++; case 5: - if (committedVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, committedVers.size())) - return false; - - commState.it = committedVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("committedVers", committedVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 6: - if (rolledbackVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, rolledbackVers.size())) - return false; - - commState.it = rolledbackVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("rolledbackVers", rolledbackVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putCacheVersion("ver", ver)) + if (!writer.writeMessage("ver", ver != null ? ver.clone() : null)) return false; - commState.idx++; + state++; } @@ -351,95 +309,51 @@ public int keysCount() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - candsByIdxBytes = commState.getByteArray("candsByIdxBytes"); + candsByIdxBytes = reader.readByteArray("candsByIdxBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - candsByKeyBytes = commState.getByteArray("candsByKeyBytes"); + candsByKeyBytes = reader.readByteArray("candsByKeyBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + committedVers = reader.readCollection("committedVers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (committedVers == null) - committedVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - committedVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + rolledbackVers = reader.readCollection("rolledbackVers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (rolledbackVers == null) - rolledbackVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - rolledbackVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - ver = commState.getCacheVersion("ver"); + ver = reader.readMessage("ver"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java index 3c94d51d39eca..a3f54c2352902 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java @@ -467,156 +467,114 @@ public GridCacheVersion[] drVersions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (drVersByIdx != null) { - if (commState.it == null) { - if (!commState.putInt(null, drVersByIdx.length)) - return false; - - commState.it = arrayIterator(drVersByIdx); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("drVersByIdx", drVersByIdx, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 10: - if (!commState.putByteArray("grpLockKeyBytes", grpLockKeyBytes)) + if (!writer.writeByteArray("grpLockKeyBytes", grpLockKeyBytes)) return false; - commState.idx++; + state++; case 11: - if (!commState.putBoolean("isInTx", isInTx)) + if (!writer.writeBoolean("isInTx", isInTx)) return false; - commState.idx++; + state++; case 12: - if (!commState.putBoolean("isInvalidate", isInvalidate)) + if (!writer.writeBoolean("isInvalidate", isInvalidate)) return false; - commState.idx++; + state++; case 13: - if (!commState.putBoolean("isRead", isRead)) + if (!writer.writeBoolean("isRead", isRead)) return false; - commState.idx++; + state++; case 14: - if (!commState.putEnum("isolation", isolation)) + if (!writer.writeEnum("isolation", isolation)) return false; - commState.idx++; + state++; case 15: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; - - commState.it = keyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("keyBytes", keyBytes, byte[].class)) + return false; - commState.idx++; + state++; case 16: - if (!commState.putCacheVersion("nearXidVer", nearXidVer)) + if (!writer.writeMessage("nearXidVer", nearXidVer != null ? nearXidVer.clone() : null)) return false; - commState.idx++; + state++; case 17: - if (!commState.putUuid("nodeId", nodeId)) + if (!writer.writeUuid("nodeId", nodeId)) return false; - commState.idx++; + state++; case 18: - if (!commState.putBoolean("partLock", partLock)) + if (!writer.writeBoolean("partLock", partLock)) return false; - commState.idx++; + state++; case 19: - if (!commState.putBooleanArray("retVals", retVals)) + if (!writer.writeBooleanArray("retVals", retVals)) return false; - commState.idx++; + state++; case 20: - if (!commState.putLong("threadId", threadId)) + if (!writer.writeLong("threadId", threadId)) return false; - commState.idx++; + state++; case 21: - if (!commState.putLong("timeout", timeout)) + if (!writer.writeLong("timeout", timeout)) return false; - commState.idx++; + state++; case 22: - if (!commState.putInt("txSize", txSize)) + if (!writer.writeInt("txSize", txSize)) return false; - commState.idx++; + state++; case 23: - if (!commState.putByteArray("writeEntriesBytes", writeEntriesBytes)) + if (!writer.writeByteArray("writeEntriesBytes", writeEntriesBytes)) return false; - commState.idx++; + state++; } @@ -626,185 +584,139 @@ public GridCacheVersion[] drVersions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (drVersByIdx == null) - drVersByIdx = new GridCacheVersion[commState.readSize]; + drVersByIdx = reader.readObjectArray("drVersByIdx", GridCacheVersion.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - drVersByIdx[i] = (GridCacheVersion)_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 9: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - grpLockKeyBytes = commState.getByteArray("grpLockKeyBytes"); + grpLockKeyBytes = reader.readByteArray("grpLockKeyBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - isInTx = commState.getBoolean("isInTx"); + isInTx = reader.readBoolean("isInTx"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - isInvalidate = commState.getBoolean("isInvalidate"); + isInvalidate = reader.readBoolean("isInvalidate"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - isRead = commState.getBoolean("isRead"); + isRead = reader.readBoolean("isRead"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - byte isolation0 = commState.getByte("isolation"); + isolation = reader.readEnum("isolation", IgniteTxIsolation.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - isolation = IgniteTxIsolation.fromOrdinal(isolation0); - - commState.idx++; + state++; case 15: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + keyBytes = reader.readCollection("keyBytes", byte[].class); - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - keyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 16: - nearXidVer = commState.getCacheVersion("nearXidVer"); + nearXidVer = reader.readMessage("nearXidVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - nodeId = commState.getUuid("nodeId"); + nodeId = reader.readUuid("nodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - partLock = commState.getBoolean("partLock"); + partLock = reader.readBoolean("partLock"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 19: - retVals = commState.getBooleanArray("retVals"); + retVals = reader.readBooleanArray("retVals"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - threadId = commState.getLong("threadId"); + threadId = reader.readLong("threadId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 21: - timeout = commState.getLong("timeout"); + timeout = reader.readLong("timeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 22: - txSize = commState.getInt("txSize"); + txSize = reader.readInt("txSize"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 23: - writeEntriesBytes = commState.getByteArray("writeEntriesBytes"); + writeEntriesBytes = reader.readByteArray("writeEntriesBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java index 32c73e6541298..ae7ac0e2ad856 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockResponse.java @@ -303,57 +303,36 @@ protected int valuesSize() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 9: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 10: - if (valBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, valBytes.size())) - return false; - - commState.it = valBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putValueBytes(null, (GridCacheValueBytes)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("valBytes", valBytes, GridCacheValueBytes.class)) + return false; - commState.idx++; + state++; } @@ -363,57 +342,35 @@ protected int valuesSize() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + valBytes = reader.readCollection("valBytes", GridCacheValueBytes.class); - if (commState.readSize >= 0) { - if (valBytes == null) - valBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheValueBytes _val = commState.getValueBytes(null); - - if (!commState.lastRead()) - return false; - - valBytes.add((GridCacheValueBytes)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java index 9a93a5ed125ba..ee4893d2013bb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishRequest.java @@ -381,138 +381,96 @@ public boolean groupLock() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putCacheVersion("baseVer", baseVer)) + if (!writer.writeMessage("baseVer", baseVer != null ? baseVer.clone() : null)) return false; - commState.idx++; + state++; case 9: - if (!commState.putBoolean("commit", commit)) + if (!writer.writeBoolean("commit", commit)) return false; - commState.idx++; + state++; case 10: - if (!commState.putCacheVersion("commitVer", commitVer)) + if (!writer.writeMessage("commitVer", commitVer != null ? commitVer.clone() : null)) return false; - commState.idx++; + state++; case 11: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 12: - if (!commState.putByteArray("grpLockKeyBytes", grpLockKeyBytes)) + if (!writer.writeByteArray("grpLockKeyBytes", grpLockKeyBytes)) return false; - commState.idx++; + state++; case 13: - if (!commState.putBoolean("invalidate", invalidate)) + if (!writer.writeBoolean("invalidate", invalidate)) return false; - commState.idx++; + state++; case 14: - if (recoveryWritesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, recoveryWritesBytes.size())) - return false; - - commState.it = recoveryWritesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("recoveryWritesBytes", recoveryWritesBytes, byte[].class)) + return false; - commState.idx++; + state++; case 15: - if (!commState.putBoolean("syncCommit", syncCommit)) + if (!writer.writeBoolean("syncCommit", syncCommit)) return false; - commState.idx++; + state++; case 16: - if (!commState.putBoolean("syncRollback", syncRollback)) + if (!writer.writeBoolean("syncRollback", syncRollback)) return false; - commState.idx++; + state++; case 17: - if (!commState.putBoolean("sys", sys)) + if (!writer.writeBoolean("sys", sys)) return false; - commState.idx++; + state++; case 18: - if (!commState.putLong("threadId", threadId)) + if (!writer.writeLong("threadId", threadId)) return false; - commState.idx++; + state++; case 19: - if (!commState.putInt("txSize", txSize)) + if (!writer.writeInt("txSize", txSize)) return false; - commState.idx++; + state++; case 20: - if (writeEntriesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, writeEntriesBytes.size())) - return false; - - commState.it = writeEntriesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("writeEntriesBytes", writeEntriesBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -522,159 +480,115 @@ public boolean groupLock() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - baseVer = commState.getCacheVersion("baseVer"); + baseVer = reader.readMessage("baseVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - commit = commState.getBoolean("commit"); + commit = reader.readBoolean("commit"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - commitVer = commState.getCacheVersion("commitVer"); + commitVer = reader.readMessage("commitVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - grpLockKeyBytes = commState.getByteArray("grpLockKeyBytes"); + grpLockKeyBytes = reader.readByteArray("grpLockKeyBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - invalidate = commState.getBoolean("invalidate"); + invalidate = reader.readBoolean("invalidate"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + recoveryWritesBytes = reader.readCollection("recoveryWritesBytes", byte[].class); - if (commState.readSize >= 0) { - if (recoveryWritesBytes == null) - recoveryWritesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - recoveryWritesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 15: - syncCommit = commState.getBoolean("syncCommit"); + syncCommit = reader.readBoolean("syncCommit"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 16: - syncRollback = commState.getBoolean("syncRollback"); + syncRollback = reader.readBoolean("syncRollback"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - sys = commState.getBoolean("sys"); + sys = reader.readBoolean("sys"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - threadId = commState.getLong("threadId"); + threadId = reader.readLong("threadId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 19: - txSize = commState.getInt("txSize"); + txSize = reader.readInt("txSize"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + writeEntriesBytes = reader.readCollection("writeEntriesBytes", byte[].class); - if (commState.readSize >= 0) { - if (writeEntriesBytes == null) - writeEntriesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - writeEntriesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java index b94daa63b9eb9..ab43a670c7497 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java @@ -97,30 +97,30 @@ public IgniteUuid futureId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 4: - if (!commState.putCacheVersion("txId", txId)) + if (!writer.writeMessage("txId", txId != null ? txId.clone() : null)) return false; - commState.idx++; + state++; } @@ -130,27 +130,27 @@ public IgniteUuid futureId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - txId = commState.getCacheVersion("txId"); + txId = reader.readMessage("txId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java index 2623188c46fc2..426c4fe606034 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareRequest.java @@ -442,144 +442,102 @@ private void writeCollection(ObjectOutput out, Collection> c /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putCacheVersion("commitVer", commitVer)) + if (!writer.writeMessage("commitVer", commitVer != null ? commitVer.clone() : null)) return false; - commState.idx++; + state++; case 9: - if (!commState.putEnum("concurrency", concurrency)) + if (!writer.writeEnum("concurrency", concurrency)) return false; - commState.idx++; + state++; case 10: - if (!commState.putByteArray("dhtVersBytes", dhtVersBytes)) + if (!writer.writeByteArray("dhtVersBytes", dhtVersBytes)) return false; - commState.idx++; + state++; case 11: - if (!commState.putByteArray("grpLockKeyBytes", grpLockKeyBytes)) + if (!writer.writeByteArray("grpLockKeyBytes", grpLockKeyBytes)) return false; - commState.idx++; + state++; case 12: - if (!commState.putBoolean("invalidate", invalidate)) + if (!writer.writeBoolean("invalidate", invalidate)) return false; - commState.idx++; + state++; case 13: - if (!commState.putEnum("isolation", isolation)) + if (!writer.writeEnum("isolation", isolation)) return false; - commState.idx++; + state++; case 14: - if (!commState.putBoolean("partLock", partLock)) + if (!writer.writeBoolean("partLock", partLock)) return false; - commState.idx++; + state++; case 15: - if (readsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, readsBytes.size())) - return false; - - commState.it = readsBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("readsBytes", readsBytes, byte[].class)) + return false; - commState.idx++; + state++; case 16: - if (!commState.putBoolean("sys", sys)) + if (!writer.writeBoolean("sys", sys)) return false; - commState.idx++; + state++; case 17: - if (!commState.putLong("threadId", threadId)) + if (!writer.writeLong("threadId", threadId)) return false; - commState.idx++; + state++; case 18: - if (!commState.putLong("timeout", timeout)) + if (!writer.writeLong("timeout", timeout)) return false; - commState.idx++; + state++; case 19: - if (!commState.putByteArray("txNodesBytes", txNodesBytes)) + if (!writer.writeByteArray("txNodesBytes", txNodesBytes)) return false; - commState.idx++; + state++; case 20: - if (!commState.putInt("txSize", txSize)) + if (!writer.writeInt("txSize", txSize)) return false; - commState.idx++; + state++; case 21: - if (writesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, writesBytes.size())) - return false; - - commState.it = writesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("writesBytes", writesBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -589,171 +547,123 @@ private void writeCollection(ObjectOutput out, Collection> c /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - commitVer = commState.getCacheVersion("commitVer"); + commitVer = reader.readMessage("commitVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - byte concurrency0 = commState.getByte("concurrency"); + concurrency = reader.readEnum("concurrency", IgniteTxConcurrency.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - concurrency = IgniteTxConcurrency.fromOrdinal(concurrency0); - - commState.idx++; + state++; case 10: - dhtVersBytes = commState.getByteArray("dhtVersBytes"); + dhtVersBytes = reader.readByteArray("dhtVersBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - grpLockKeyBytes = commState.getByteArray("grpLockKeyBytes"); + grpLockKeyBytes = reader.readByteArray("grpLockKeyBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - invalidate = commState.getBoolean("invalidate"); + invalidate = reader.readBoolean("invalidate"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - byte isolation0 = commState.getByte("isolation"); + isolation = reader.readEnum("isolation", IgniteTxIsolation.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - isolation = IgniteTxIsolation.fromOrdinal(isolation0); - - commState.idx++; + state++; case 14: - partLock = commState.getBoolean("partLock"); + partLock = reader.readBoolean("partLock"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 15: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (readsBytes == null) - readsBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - readsBytes.add((byte[])_val); + readsBytes = reader.readCollection("readsBytes", byte[].class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 16: - sys = commState.getBoolean("sys"); + sys = reader.readBoolean("sys"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - threadId = commState.getLong("threadId"); + threadId = reader.readLong("threadId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - timeout = commState.getLong("timeout"); + timeout = reader.readLong("timeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 19: - txNodesBytes = commState.getByteArray("txNodesBytes"); + txNodesBytes = reader.readByteArray("txNodesBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - txSize = commState.getInt("txSize"); + txSize = reader.readInt("txSize"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 21: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + writesBytes = reader.readCollection("writesBytes", byte[].class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (writesBytes == null) - writesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - writesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java index 057916c7fd154..cd475133afc94 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxPrepareResponse.java @@ -174,30 +174,30 @@ public void candidates(Map>> cands) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (!commState.putByteArray("candsBytes", candsBytes)) + if (!writer.writeByteArray("candsBytes", candsBytes)) return false; - commState.idx++; + state++; case 9: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; } @@ -207,27 +207,27 @@ public void candidates(Map>> cands) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - candsBytes = commState.getByteArray("candsBytes"); + candsBytes = reader.readByteArray("candsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java index d865208ae3235..9c3f6abd61b4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java @@ -138,45 +138,24 @@ public void addKey(K key, byte[] bytes, GridCacheContext ctx) throws Ignit /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 8: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; + if (!writer.writeCollection("keyBytes", keyBytes, byte[].class)) + return false; - commState.it = keyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } - - commState.idx++; + state++; } @@ -186,41 +165,19 @@ public void addKey(K key, byte[] bytes, GridCacheContext ctx) throws Ignit /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 8: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - keyBytes.add((byte[])_val); - - commState.readItems++; - } - } + keyBytes = reader.readCollection("keyBytes", byte[].class); - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java index 1ba375351b58d..3534fd30b37d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java @@ -87,24 +87,24 @@ public GridDhtAffinityAssignmentRequest(int cacheId, long topVer) { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -113,19 +113,19 @@ public GridDhtAffinityAssignmentRequest(int cacheId, long topVer) { /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java index 14d70e8d17a3b..4a44303cedf00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java @@ -128,30 +128,30 @@ public List> affinityAssignment() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("affAssignmentBytes", affAssignmentBytes)) + if (!writer.writeByteArray("affAssignmentBytes", affAssignmentBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -160,27 +160,27 @@ public List> affinityAssignment() { /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - affAssignmentBytes = commState.getByteArray("affAssignmentBytes"); + affAssignmentBytes = reader.readByteArray("affAssignmentBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java index 7a997209b589f..a9a93f764b87c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java @@ -372,93 +372,72 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 24: - if (!commState.putLong("accessTtl", accessTtl)) + if (!writer.writeLong("accessTtl", accessTtl)) return false; - commState.idx++; + state++; case 25: - if (!commState.putBitSet("invalidateEntries", invalidateEntries)) + if (!writer.writeBitSet("invalidateEntries", invalidateEntries)) return false; - commState.idx++; + state++; case 26: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 27: - if (nearKeyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearKeyBytes.size())) - return false; - - commState.it = nearKeyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearKeyBytes", nearKeyBytes, byte[].class)) + return false; - commState.idx++; + state++; case 28: - if (!commState.putByteArray("ownedBytes", ownedBytes)) + if (!writer.writeByteArray("ownedBytes", ownedBytes)) return false; - commState.idx++; + state++; case 29: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeBitSet("preloadKeys", preloadKeys)) return false; - commState.idx++; + state++; case 30: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 31: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 32: - if (!commState.putBitSet("preloadKeys", preloadKeys)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -468,105 +447,83 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 24: - accessTtl = commState.getLong("accessTtl"); + accessTtl = reader.readLong("accessTtl"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 25: - invalidateEntries = commState.getBitSet("invalidateEntries"); + invalidateEntries = reader.readBitSet("invalidateEntries"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 26: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 27: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nearKeyBytes = reader.readCollection("nearKeyBytes", byte[].class); - if (commState.readSize >= 0) { - if (nearKeyBytes == null) - nearKeyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearKeyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 28: - ownedBytes = commState.getByteArray("ownedBytes"); + ownedBytes = reader.readByteArray("ownedBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 29: - topVer = commState.getLong("topVer"); + preloadKeys = reader.readBitSet("preloadKeys"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 30: - subjId = commState.getUuid("subjId"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 31: - taskNameHash = commState.getInt("taskNameHash"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 32: - preloadKeys = commState.getBitSet("preloadKeys"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java index f301a8fb93f14..2f28b8991d88d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java @@ -225,105 +225,42 @@ public Collection> preloadEntries() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 11: - if (invalidParts != null) { - if (commState.it == null) { - if (!commState.putInt(null, invalidParts.size())) - return false; - - commState.it = invalidParts.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("invalidParts", invalidParts, int.class)) + return false; - commState.idx++; + state++; case 12: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 13: - if (nearEvictedBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearEvictedBytes.size())) - return false; - - commState.it = nearEvictedBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearEvictedBytes", nearEvictedBytes, byte[].class)) + return false; - commState.idx++; + state++; case 14: - if (preloadEntriesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, preloadEntriesBytes.size())) - return false; - - commState.it = preloadEntriesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("preloadEntriesBytes", preloadEntriesBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -333,109 +270,43 @@ public Collection> preloadEntries() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 11: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (invalidParts == null) - invalidParts = new HashSet<>(commState.readSize); + invalidParts = reader.readCollection("invalidParts", int.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - invalidParts.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 12: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearEvictedBytes == null) - nearEvictedBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearEvictedBytes.add((byte[])_val); + nearEvictedBytes = reader.readCollection("nearEvictedBytes", byte[].class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 14: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (preloadEntriesBytes == null) - preloadEntriesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); + preloadEntriesBytes = reader.readCollection("preloadEntriesBytes", byte[].class); - if (!commState.lastRead()) - return false; - - preloadEntriesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java index f814c5d597ac8..adc6df97f7e04 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java @@ -382,138 +382,96 @@ public GridLongList nearTtls() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 21: - if (!commState.putEnum("isolation", isolation)) + if (!writer.writeEnum("isolation", isolation)) return false; - commState.idx++; + state++; case 22: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 23: - if (!commState.putUuid("nearNodeId", nearNodeId)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; - commState.idx++; + state++; case 24: - if (!commState.putLongList("nearTtls", nearTtls)) + if (!writer.writeMessage("nearTtls", nearTtls != null ? nearTtls.clone() : null)) return false; - commState.idx++; + state++; case 25: - if (nearWritesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearWritesBytes.size())) - return false; - - commState.it = nearWritesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearWritesBytes", nearWritesBytes, byte[].class)) + return false; - commState.idx++; + state++; case 26: - if (!commState.putBoolean("onePhaseCommit", onePhaseCommit)) + if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit)) return false; - commState.idx++; + state++; case 27: - if (pendingVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, pendingVers.size())) - return false; - - commState.it = pendingVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("pendingVers", pendingVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 28: - if (!commState.putBoolean("sysInvalidate", sysInvalidate)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 29: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeBoolean("sysInvalidate", sysInvalidate)) return false; - commState.idx++; + state++; case 30: - if (!commState.putLongList("ttls", ttls)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 31: - if (!commState.putCacheVersion("writeVer", writeVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 32: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeMessage("ttls", ttls != null ? ttls.clone() : null)) return false; - commState.idx++; + state++; case 33: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("writeVer", writeVer != null ? writeVer.clone() : null)) return false; - commState.idx++; + state++; } @@ -523,161 +481,115 @@ public GridLongList nearTtls() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 21: - byte isolation0 = commState.getByte("isolation"); + isolation = reader.readEnum("isolation", IgniteTxIsolation.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - isolation = IgniteTxIsolation.fromOrdinal(isolation0); - - commState.idx++; + state++; case 22: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 23: - nearNodeId = commState.getUuid("nearNodeId"); + nearNodeId = reader.readUuid("nearNodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 24: - nearTtls = commState.getLongList("nearTtls"); + nearTtls = reader.readMessage("nearTtls"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 25: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nearWritesBytes = reader.readCollection("nearWritesBytes", byte[].class); - if (commState.readSize >= 0) { - if (nearWritesBytes == null) - nearWritesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearWritesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 26: - onePhaseCommit = commState.getBoolean("onePhaseCommit"); + onePhaseCommit = reader.readBoolean("onePhaseCommit"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 27: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + pendingVers = reader.readCollection("pendingVers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (pendingVers == null) - pendingVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - pendingVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 28: - sysInvalidate = commState.getBoolean("sysInvalidate"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 29: - topVer = commState.getLong("topVer"); + sysInvalidate = reader.readBoolean("sysInvalidate"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 30: - ttls = commState.getLongList("ttls"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 31: - writeVer = commState.getCacheVersion("writeVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 32: - subjId = commState.getUuid("subjId"); + ttls = reader.readMessage("ttls"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 33: - taskNameHash = commState.getInt("taskNameHash"); + writeVer = reader.readMessage("writeVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java index 590f06a7153da..c8b2046a9cd69 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishResponse.java @@ -90,24 +90,24 @@ public IgniteUuid miniId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 5: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; } @@ -117,19 +117,19 @@ public IgniteUuid miniId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 5: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java index d7af7808c11a1..a57827c22c083 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java @@ -346,111 +346,90 @@ public Map, GridCacheVersion> owned() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 22: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 23: - if (!commState.putBitSet("invalidateNearEntries", invalidateNearEntries)) + if (!writer.writeBitSet("invalidateNearEntries", invalidateNearEntries)) return false; - commState.idx++; + state++; case 24: - if (!commState.putBoolean("last", last)) + if (!writer.writeBoolean("last", last)) return false; - commState.idx++; + state++; case 25: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 26: - if (!commState.putUuid("nearNodeId", nearNodeId)) + if (!writer.writeUuid("nearNodeId", nearNodeId)) return false; - commState.idx++; + state++; case 27: - if (nearWritesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearWritesBytes.size())) - return false; - - commState.it = nearWritesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearWritesBytes", nearWritesBytes, byte[].class)) + return false; - commState.idx++; + state++; case 28: - if (!commState.putCacheVersion("nearXidVer", nearXidVer)) + if (!writer.writeMessage("nearXidVer", nearXidVer != null ? nearXidVer.clone() : null)) return false; - commState.idx++; + state++; case 29: - if (!commState.putByteArray("ownedBytes", ownedBytes)) + if (!writer.writeByteArray("ownedBytes", ownedBytes)) return false; - commState.idx++; + state++; case 30: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeBitSet("preloadKeys", preloadKeys)) return false; - commState.idx++; + state++; case 31: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 32: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 33: - if (!commState.putBitSet("preloadKeys", preloadKeys)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -460,129 +439,107 @@ public Map, GridCacheVersion> owned() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 22: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 23: - invalidateNearEntries = commState.getBitSet("invalidateNearEntries"); + invalidateNearEntries = reader.readBitSet("invalidateNearEntries"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 24: - last = commState.getBoolean("last"); + last = reader.readBoolean("last"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 25: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 26: - nearNodeId = commState.getUuid("nearNodeId"); + nearNodeId = reader.readUuid("nearNodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 27: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nearWritesBytes = reader.readCollection("nearWritesBytes", byte[].class); - if (commState.readSize >= 0) { - if (nearWritesBytes == null) - nearWritesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearWritesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 28: - nearXidVer = commState.getCacheVersion("nearXidVer"); + nearXidVer = reader.readMessage("nearXidVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 29: - ownedBytes = commState.getByteArray("ownedBytes"); + ownedBytes = reader.readByteArray("ownedBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 30: - topVer = commState.getLong("topVer"); + preloadKeys = reader.readBitSet("preloadKeys"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 31: - subjId = commState.getUuid("subjId"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 32: - taskNameHash = commState.getInt("taskNameHash"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 33: - preloadKeys = commState.getBitSet("preloadKeys"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java index fd6c0b44ed40f..319639949e2a3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareResponse.java @@ -232,111 +232,48 @@ public void addPreloadEntry(GridCacheEntryInfo info) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 10: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 11: - if (invalidParts != null) { - if (commState.it == null) { - if (!commState.putInt(null, invalidParts.size())) - return false; - - commState.it = invalidParts.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("invalidParts", invalidParts, int.class)) + return false; - commState.idx++; + state++; case 12: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 13: - if (nearEvictedBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearEvictedBytes.size())) - return false; - - commState.it = nearEvictedBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearEvictedBytes", nearEvictedBytes, byte[].class)) + return false; - commState.idx++; + state++; case 14: - if (preloadEntriesBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, preloadEntriesBytes.size())) - return false; - - commState.it = preloadEntriesBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("preloadEntriesBytes", preloadEntriesBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -346,117 +283,51 @@ public void addPreloadEntry(GridCacheEntryInfo info) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 10: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (invalidParts == null) - invalidParts = new ArrayList<>(commState.readSize); + invalidParts = reader.readCollection("invalidParts", int.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - invalidParts.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 12: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearEvictedBytes == null) - nearEvictedBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearEvictedBytes.add((byte[])_val); + nearEvictedBytes = reader.readCollection("nearEvictedBytes", byte[].class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 14: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (preloadEntriesBytes == null) - preloadEntriesBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); + preloadEntriesBytes = reader.readCollection("preloadEntriesBytes", byte[].class); - if (!commState.lastRead()) - return false; - - preloadEntriesBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java index 19a9dce6b4452..7c381d6b37e83 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtUnlockRequest.java @@ -126,45 +126,24 @@ public void addNearKey(K key, byte[] keyBytes, GridCacheSharedContext ctx) /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 9: - if (nearKeyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearKeyBytes.size())) - return false; + if (!writer.writeCollection("nearKeyBytes", nearKeyBytes, byte[].class)) + return false; - commState.it = nearKeyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } - - commState.idx++; + state++; } @@ -174,41 +153,19 @@ public void addNearKey(K key, byte[] keyBytes, GridCacheSharedContext ctx) /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 9: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearKeyBytes == null) - nearKeyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearKeyBytes.add((byte[])_val); - - commState.readItems++; - } - } + nearKeyBytes = reader.readCollection("nearKeyBytes", byte[].class); - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java index 07ba0bbcd3d6e..955bb4f55ba45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicDeferredUpdateResponse.java @@ -94,45 +94,24 @@ public Collection futureVersions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (futVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, futVers.size())) - return false; + if (!writer.writeCollection("futVers", futVers, GridCacheVersion.class)) + return false; - commState.it = futVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } - - commState.idx++; + state++; } @@ -142,41 +121,19 @@ public Collection futureVersions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (futVers == null) - futVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - futVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } + futVers = reader.readCollection("futVers", GridCacheVersion.class); - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java index c69fc6e6036ca..6b59e829d7a4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java @@ -728,306 +728,138 @@ public long nearExpireTime(int idx) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putLongList("drExpireTimes", drExpireTimes)) + if (!writer.writeMessage("drExpireTimes", drExpireTimes != null ? drExpireTimes.clone() : null)) return false; - commState.idx++; + state++; case 4: - if (drVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, drVers.size())) - return false; - - commState.it = drVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("drVers", drVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 5: - if (!commState.putCacheVersion("futVer", futVer)) + if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, byte[].class)) return false; - commState.idx++; + state++; case 6: - if (invokeArgsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, invokeArgsBytes.length)) - return false; - - commState.it = arrayIterator(invokeArgsBytes); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeBoolean("forceTransformBackups", forceTransformBackups)) + return false; - commState.idx++; + state++; case 7: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; - - commState.it = keyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMessage("futVer", futVer != null ? futVer.clone() : null)) + return false; - commState.idx++; + state++; case 8: - if (!commState.putLongList("nearExpireTimes", nearExpireTimes)) + if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, byte[].class)) return false; - commState.idx++; + state++; case 9: - if (!commState.putLongList("nearTtls", nearTtls)) + if (!writer.writeCollection("keyBytes", keyBytes, byte[].class)) return false; - commState.idx++; + state++; case 10: - if (!commState.putUuid("nodeId", nodeId)) + if (!writer.writeCollection("nearEntryProcessorsBytes", nearEntryProcessorsBytes, byte[].class)) return false; - commState.idx++; + state++; case 11: - if (!commState.putEnum("syncMode", syncMode)) + if (!writer.writeMessage("nearExpireTimes", nearExpireTimes != null ? nearExpireTimes.clone() : null)) return false; - commState.idx++; + state++; case 12: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeCollection("nearKeyBytes", nearKeyBytes, byte[].class)) return false; - commState.idx++; + state++; case 13: - if (!commState.putLongList("ttls", ttls)) + if (!writer.writeMessage("nearTtls", nearTtls != null ? nearTtls.clone() : null)) return false; - commState.idx++; + state++; case 14: - if (valBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, valBytes.size())) - return false; - - commState.it = valBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putValueBytes(null, (GridCacheValueBytes)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearValBytes", nearValBytes, GridCacheValueBytes.class)) + return false; - commState.idx++; + state++; case 15: - if (!commState.putCacheVersion("writeVer", writeVer)) + if (!writer.writeUuid("nodeId", nodeId)) return false; - commState.idx++; + state++; case 16: - if (nearKeyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearKeyBytes.size())) - return false; - - commState.it = nearKeyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeUuid("subjId", subjId)) + return false; - commState.idx++; + state++; case 17: - if (nearValBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearValBytes.size())) - return false; - - commState.it = nearValBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putValueBytes(null, (GridCacheValueBytes)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeEnum("syncMode", syncMode)) + return false; - commState.idx++; + state++; case 18: - if (entryProcessorsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, entryProcessorsBytes.size())) - return false; - - commState.it = entryProcessorsBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; - commState.idx++; + state++; case 19: - if (!commState.putBoolean("forceTransformBackups", forceTransformBackups)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 20: - if (nearEntryProcessorsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearEntryProcessorsBytes.size())) - return false; - - commState.it = nearEntryProcessorsBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMessage("ttls", ttls != null ? ttls.clone() : null)) + return false; - commState.idx++; + state++; case 21: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeCollection("valBytes", valBytes, GridCacheValueBytes.class)) return false; - commState.idx++; + state++; case 22: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("writeVer", writeVer != null ? writeVer.clone() : null)) return false; - commState.idx++; + state++; } @@ -1037,349 +869,171 @@ public long nearExpireTime(int idx) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - drExpireTimes = commState.getLongList("drExpireTimes"); + drExpireTimes = reader.readMessage("drExpireTimes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (drVers == null) - drVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - drVers.add((GridCacheVersion)_val); + drVers = reader.readCollection("drVers", GridCacheVersion.class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - futVer = commState.getCacheVersion("futVer"); + entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (invokeArgsBytes == null) - invokeArgsBytes = new byte[commState.readSize][]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - invokeArgsBytes[i] = (byte[])_val; + forceTransformBackups = reader.readBoolean("forceTransformBackups"); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; + futVer = reader.readMessage("futVer"); - keyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 8: - nearExpireTimes = commState.getLongList("nearExpireTimes"); + invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - nearTtls = commState.getLongList("nearTtls"); + keyBytes = reader.readCollection("keyBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - nodeId = commState.getUuid("nodeId"); + nearEntryProcessorsBytes = reader.readCollection("nearEntryProcessorsBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - byte syncMode0 = commState.getByte("syncMode"); + nearExpireTimes = reader.readMessage("nearExpireTimes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncMode0); - - commState.idx++; + state++; case 12: - topVer = commState.getLong("topVer"); + nearKeyBytes = reader.readCollection("nearKeyBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - ttls = commState.getLongList("ttls"); + nearTtls = reader.readMessage("nearTtls"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (valBytes == null) - valBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheValueBytes _val = commState.getValueBytes(null); + nearValBytes = reader.readCollection("nearValBytes", GridCacheValueBytes.class); - if (!commState.lastRead()) - return false; - - valBytes.add((GridCacheValueBytes)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 15: - writeVer = commState.getCacheVersion("writeVer"); + nodeId = reader.readUuid("nodeId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 16: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearKeyBytes == null) - nearKeyBytes = new ArrayList<>(commState.readSize); + subjId = reader.readUuid("subjId"); - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearKeyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 17: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearValBytes == null) - nearValBytes = new ArrayList<>(commState.readSize); + syncMode = reader.readEnum("syncMode", CacheWriteSynchronizationMode.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheValueBytes _val = commState.getValueBytes(null); - - if (!commState.lastRead()) - return false; - - nearValBytes.add((GridCacheValueBytes)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 18: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + taskNameHash = reader.readInt("taskNameHash"); - if (commState.readSize >= 0) { - if (entryProcessorsBytes == null) - entryProcessorsBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - entryProcessorsBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 19: - forceTransformBackups = commState.getBoolean("forceTransformBackups"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + ttls = reader.readMessage("ttls"); - if (commState.readSize >= 0) { - if (nearEntryProcessorsBytes == null) - nearEntryProcessorsBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearEntryProcessorsBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 21: - subjId = commState.getUuid("subjId"); + valBytes = reader.readCollection("valBytes", GridCacheValueBytes.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 22: - taskNameHash = commState.getInt("taskNameHash"); + writeVer = reader.readMessage("writeVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java index afd013c8fea8f..256c4b6cc2c79 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateResponse.java @@ -205,63 +205,42 @@ public void addNearEvicted(K key, @Nullable byte[] bytes) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("failedKeysBytes", failedKeysBytes)) + if (!writer.writeByteArray("failedKeysBytes", failedKeysBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putCacheVersion("futVer", futVer)) + if (!writer.writeMessage("futVer", futVer != null ? futVer.clone() : null)) return false; - commState.idx++; + state++; case 6: - if (nearEvictedBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearEvictedBytes.size())) - return false; - - commState.it = nearEvictedBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearEvictedBytes", nearEvictedBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -271,65 +250,43 @@ public void addNearEvicted(K key, @Nullable byte[] bytes) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - failedKeysBytes = commState.getByteArray("failedKeysBytes"); + failedKeysBytes = reader.readByteArray("failedKeysBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - futVer = commState.getCacheVersion("futVer"); + futVer = reader.readMessage("futVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + nearEvictedBytes = reader.readCollection("nearEvictedBytes", byte[].class); - if (commState.readSize >= 0) { - if (nearEvictedBytes == null) - nearEvictedBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - nearEvictedBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java index f700649dd4924..cf234512be39f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java @@ -587,237 +587,132 @@ public void forceTransformBackups(boolean forceTransformBackups) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putLongList("drExpireTimes", drExpireTimes)) + if (!writer.writeMessage("drExpireTimes", drExpireTimes != null ? drExpireTimes.clone() : null)) return false; - commState.idx++; + state++; case 4: - if (!commState.putLongList("drTtls", drTtls)) + if (!writer.writeMessage("drTtls", drTtls != null ? drTtls.clone() : null)) return false; - commState.idx++; + state++; case 5: - if (drVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, drVers.size())) - return false; - - commState.it = drVers.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("drVers", drVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 6: - if (!commState.putByteArray("expiryPlcBytes", expiryPlcBytes)) + if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes)) return false; - commState.idx++; + state++; case 7: - if (!commState.putBoolean("fastMap", fastMap)) + if (!writer.writeBoolean("fastMap", fastMap)) return false; - commState.idx++; + state++; case 8: - if (filterBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, filterBytes.length)) - return false; - - commState.it = arrayIterator(filterBytes); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("filterBytes", filterBytes, byte[].class)) + return false; - commState.idx++; + state++; case 9: - if (!commState.putCacheVersion("futVer", futVer)) + if (!writer.writeBoolean("forceTransformBackups", forceTransformBackups)) return false; - commState.idx++; + state++; case 10: - if (!commState.putBoolean("hasPrimary", hasPrimary)) + if (!writer.writeMessage("futVer", futVer != null ? futVer.clone() : null)) return false; - commState.idx++; + state++; case 11: - if (invokeArgsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, invokeArgsBytes.length)) - return false; - - commState.it = arrayIterator(invokeArgsBytes); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeBoolean("hasPrimary", hasPrimary)) + return false; - commState.idx++; + state++; case 12: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; - - commState.it = keyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, byte[].class)) + return false; - commState.idx++; + state++; case 13: - if (!commState.putEnum("op", op)) + if (!writer.writeCollection("keyBytes", keyBytes, byte[].class)) return false; - commState.idx++; + state++; case 14: - if (!commState.putBoolean("retval", retval)) + if (!writer.writeEnum("op", op)) return false; - commState.idx++; + state++; case 15: - if (!commState.putEnum("syncMode", syncMode)) + if (!writer.writeBoolean("retval", retval)) return false; - commState.idx++; + state++; case 16: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 17: - if (!commState.putCacheVersion("updateVer", updateVer)) + if (!writer.writeEnum("syncMode", syncMode)) return false; - commState.idx++; + state++; case 18: - if (valBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, valBytes.size())) - return false; - - commState.it = valBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putValueBytes(null, (GridCacheValueBytes)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeInt("taskNameHash", taskNameHash)) + return false; - commState.idx++; + state++; case 19: - if (!commState.putBoolean("forceTransformBackups", forceTransformBackups)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 20: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeMessage("updateVer", updateVer != null ? updateVer.clone() : null)) return false; - commState.idx++; + state++; case 21: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeCollection("valBytes", valBytes, GridCacheValueBytes.class)) return false; - commState.idx++; + state++; } @@ -827,277 +722,163 @@ public void forceTransformBackups(boolean forceTransformBackups) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - drExpireTimes = commState.getLongList("drExpireTimes"); + drExpireTimes = reader.readMessage("drExpireTimes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - drTtls = commState.getLongList("drTtls"); + drTtls = reader.readMessage("drTtls"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (drVers == null) - drVers = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; + drVers = reader.readCollection("drVers", GridCacheVersion.class); - drVers.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 6: - expiryPlcBytes = commState.getByteArray("expiryPlcBytes"); + expiryPlcBytes = reader.readByteArray("expiryPlcBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - fastMap = commState.getBoolean("fastMap"); + fastMap = reader.readBoolean("fastMap"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (filterBytes == null) - filterBytes = new byte[commState.readSize][]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - filterBytes[i] = (byte[])_val; - - commState.readItems++; - } - } + filterBytes = reader.readObjectArray("filterBytes", byte[].class); - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 9: - futVer = commState.getCacheVersion("futVer"); + forceTransformBackups = reader.readBoolean("forceTransformBackups"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - hasPrimary = commState.getBoolean("hasPrimary"); + futVer = reader.readMessage("futVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (invokeArgsBytes == null) - invokeArgsBytes = new byte[commState.readSize][]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); + hasPrimary = reader.readBoolean("hasPrimary"); - if (!commState.lastRead()) - return false; - - invokeArgsBytes[i] = (byte[])_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 12: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", byte[].class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - keyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 13: - byte op0 = commState.getByte("op"); + keyBytes = reader.readCollection("keyBytes", byte[].class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - op = GridCacheOperation.fromOrdinal(op0); - - commState.idx++; + state++; case 14: - retval = commState.getBoolean("retval"); + op = reader.readEnum("op", GridCacheOperation.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 15: - byte syncMode0 = commState.getByte("syncMode"); + retval = reader.readBoolean("retval"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - syncMode = CacheWriteSynchronizationMode.fromOrdinal(syncMode0); - - commState.idx++; + state++; case 16: - topVer = commState.getLong("topVer"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - updateVer = commState.getCacheVersion("updateVer"); + syncMode = reader.readEnum("syncMode", CacheWriteSynchronizationMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (valBytes == null) - valBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheValueBytes _val = commState.getValueBytes(null); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) - return false; - - valBytes.add((GridCacheValueBytes)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 19: - forceTransformBackups = commState.getBoolean("forceTransformBackups"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - subjId = commState.getUuid("subjId"); + updateVer = reader.readMessage("updateVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 21: - taskNameHash = commState.getInt("taskNameHash"); + valBytes = reader.readCollection("valBytes", GridCacheValueBytes.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java index d91e8001818df..23bca9b61b545 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java @@ -455,147 +455,84 @@ public synchronized void addFailedKeys(Collection keys, Throwable e) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("failedKeysBytes", failedKeysBytes)) + if (!writer.writeByteArray("failedKeysBytes", failedKeysBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putCacheVersion("futVer", futVer)) + if (!writer.writeMessage("futVer", futVer != null ? futVer.clone() : null)) return false; - commState.idx++; + state++; case 6: - if (!commState.putLongList("nearExpireTimes", nearExpireTimes)) + if (!writer.writeMessage("nearExpireTimes", nearExpireTimes != null ? nearExpireTimes.clone() : null)) return false; - commState.idx++; + state++; case 7: - if (!commState.putLongList("nearTtls", nearTtls)) + if (!writer.writeCollection("nearSkipIdxs", nearSkipIdxs, int.class)) return false; - commState.idx++; + state++; case 8: - if (!commState.putByteArray("remapKeysBytes", remapKeysBytes)) + if (!writer.writeMessage("nearTtls", nearTtls != null ? nearTtls.clone() : null)) return false; - commState.idx++; + state++; case 9: - if (!commState.putByteArray("retValBytes", retValBytes)) + if (!writer.writeCollection("nearValBytes", nearValBytes, GridCacheValueBytes.class)) return false; - commState.idx++; + state++; case 10: - if (nearSkipIdxs != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearSkipIdxs.size())) - return false; - - commState.it = nearSkipIdxs.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("nearValsIdxs", nearValsIdxs, int.class)) + return false; - commState.idx++; + state++; case 11: - if (nearValBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearValBytes.size())) - return false; - - commState.it = nearValBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putValueBytes(null, (GridCacheValueBytes)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMessage("nearVer", nearVer != null ? nearVer.clone() : null)) + return false; - commState.idx++; + state++; case 12: - if (nearValsIdxs != null) { - if (commState.it == null) { - if (!commState.putInt(null, nearValsIdxs.size())) - return false; - - commState.it = nearValsIdxs.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeByteArray("remapKeysBytes", remapKeysBytes)) + return false; - commState.idx++; + state++; case 13: - if (!commState.putCacheVersion("nearVer", nearVer)) + if (!writer.writeByteArray("retValBytes", retValBytes)) return false; - commState.idx++; + state++; } @@ -605,165 +542,99 @@ public synchronized void addFailedKeys(Collection keys, Throwable e) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - failedKeysBytes = commState.getByteArray("failedKeysBytes"); + failedKeysBytes = reader.readByteArray("failedKeysBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - futVer = commState.getCacheVersion("futVer"); + futVer = reader.readMessage("futVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - nearExpireTimes = commState.getLongList("nearExpireTimes"); + nearExpireTimes = reader.readMessage("nearExpireTimes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - nearTtls = commState.getLongList("nearTtls"); + nearSkipIdxs = reader.readCollection("nearSkipIdxs", int.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - remapKeysBytes = commState.getByteArray("remapKeysBytes"); + nearTtls = reader.readMessage("nearTtls"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - retValBytes = commState.getByteArray("retValBytes"); + nearValBytes = reader.readCollection("nearValBytes", GridCacheValueBytes.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearSkipIdxs == null) - nearSkipIdxs = new ArrayList<>(commState.readSize); + nearValsIdxs = reader.readCollection("nearValsIdxs", int.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - nearSkipIdxs.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 11: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearValBytes == null) - nearValBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheValueBytes _val = commState.getValueBytes(null); - - if (!commState.lastRead()) - return false; - - nearValBytes.add((GridCacheValueBytes)_val); + nearVer = reader.readMessage("nearVer"); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 12: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (nearValsIdxs == null) - nearValsIdxs = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); + remapKeysBytes = reader.readByteArray("remapKeysBytes"); - if (!commState.lastRead()) - return false; - - nearValsIdxs.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 13: - nearVer = commState.getCacheVersion("nearVer"); + retValBytes = reader.readByteArray("retValBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java index 66a6ad156fb6c..60c4a9a2a8622 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java @@ -187,63 +187,42 @@ private int keyCount() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 4: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; - - commState.it = keyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("keyBytes", keyBytes, byte[].class)) + return false; - commState.idx++; + state++; case 5: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 6: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -253,65 +232,43 @@ private int keyCount() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + keyBytes = reader.readCollection("keyBytes", byte[].class); - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - keyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java index 626cb1fe93c9e..ff8cdca66ba00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java @@ -191,63 +191,42 @@ public void addInfo(GridCacheEntryInfo info) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("infosBytes", infosBytes)) + if (!writer.writeByteArray("infosBytes", infosBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 6: - if (missedKeyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, missedKeyBytes.size())) - return false; - - commState.it = missedKeyBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("missedKeyBytes", missedKeyBytes, byte[].class)) + return false; - commState.idx++; + state++; } @@ -257,65 +236,43 @@ public void addInfo(GridCacheEntryInfo info) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - infosBytes = commState.getByteArray("infosBytes"); + infosBytes = reader.readByteArray("infosBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + missedKeyBytes = reader.readCollection("missedKeyBytes", byte[].class); - if (commState.readSize >= 0) { - if (missedKeyBytes == null) - missedKeyBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - missedKeyBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java index 59eb2bdfe8b67..409522c808034 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java @@ -217,75 +217,54 @@ void workerId(int workerId) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (parts != null) { - if (commState.it == null) { - if (!commState.putInt(null, parts.size())) - return false; - - commState.it = parts.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("parts", parts, int.class)) + return false; - commState.idx++; + state++; case 4: - if (!commState.putLong("timeout", timeout)) + if (!writer.writeLong("timeout", timeout)) return false; - commState.idx++; + state++; case 5: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 6: - if (!commState.putByteArray("topicBytes", topicBytes)) + if (!writer.writeByteArray("topicBytes", topicBytes)) return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("updateSeq", updateSeq)) + if (!writer.writeLong("updateSeq", updateSeq)) return false; - commState.idx++; + state++; case 8: - if (!commState.putInt("workerId", workerId)) + if (!writer.writeInt("workerId", workerId)) return false; - commState.idx++; + state++; } @@ -295,81 +274,59 @@ void workerId(int workerId) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + parts = reader.readCollection("parts", int.class); - if (commState.readSize >= 0) { - if (parts == null) - parts = new HashSet<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - parts.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 4: - timeout = commState.getLong("timeout"); + timeout = reader.readLong("timeout"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - topicBytes = commState.getByteArray("topicBytes"); + topicBytes = reader.readByteArray("topicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - updateSeq = commState.getLong("updateSeq"); + updateSeq = reader.readLong("updateSeq"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - workerId = commState.getInt("workerId"); + workerId = reader.readInt("workerId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java index dc9d26c1bce01..622f639afbbe3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionExchangeId.java @@ -147,12 +147,71 @@ public boolean isLeft() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeInt("evt", evt)) + return false; + + state++; + + case 1: + if (!writer.writeUuid("nodeId", nodeId)) + return false; + + state++; + + case 2: + if (!writer.writeLong("topVer", topVer)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + evt = reader.readInt("evt"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + nodeId = reader.readUuid("nodeId"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 2: + topVer = reader.readLong("topVer"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -162,12 +221,20 @@ public boolean isLeft() { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridDhtPartitionExchangeId _clone = new GridDhtPartitionExchangeId(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridDhtPartitionExchangeId _clone = (GridDhtPartitionExchangeId)_msg; + + _clone.nodeId = nodeId; + _clone.evt = evt; + _clone.topVer = topVer; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java index 8b953a986e1b5..4835e771e40a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java @@ -302,96 +302,54 @@ public int size() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putBoolean("ack", ack)) + if (!writer.writeBoolean("ack", ack)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("infoBytes", infoBytes)) + if (!writer.writeByteArray("infoBytes", infoBytes)) return false; - commState.idx++; + state++; case 5: - if (last != null) { - if (commState.it == null) { - if (!commState.putInt(null, last.size())) - return false; - - commState.it = last.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("last", last, int.class)) + return false; - commState.idx++; + state++; case 6: - if (missed != null) { - if (commState.it == null) { - if (!commState.putInt(null, missed.size())) - return false; - - commState.it = missed.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("missed", missed, int.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("updateSeq", updateSeq)) + if (!writer.writeLong("updateSeq", updateSeq)) return false; - commState.idx++; + state++; case 8: - if (!commState.putInt("workerId", workerId)) + if (!writer.writeInt("workerId", workerId)) return false; - commState.idx++; + state++; } @@ -401,103 +359,59 @@ public int size() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - ack = commState.getBoolean("ack"); + ack = reader.readBoolean("ack"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - infoBytes = commState.getByteArray("infoBytes"); + infoBytes = reader.readByteArray("infoBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + last = reader.readCollection("last", int.class); - if (commState.readSize >= 0) { - if (last == null) - last = new HashSet<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - last.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + missed = reader.readCollection("missed", int.class); - if (commState.readSize >= 0) { - if (missed == null) - missed = new HashSet<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - missed.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - updateSeq = commState.getLong("updateSeq"); + updateSeq = reader.readLong("updateSeq"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - workerId = commState.getInt("workerId"); + workerId = reader.readInt("workerId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java index b8905f9c0b1d1..961da85732d81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java @@ -87,30 +87,30 @@ public GridDhtPartitionExchangeId exchangeId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putDhtPartitionExchangeId("exchId", exchId)) + if (!writer.writeMessage("exchId", exchId != null ? exchId.clone() : null)) return false; - commState.idx++; + state++; case 4: - if (!commState.putCacheVersion("lastVer", lastVer)) + if (!writer.writeMessage("lastVer", lastVer != null ? lastVer.clone() : null)) return false; - commState.idx++; + state++; } @@ -120,27 +120,27 @@ public GridDhtPartitionExchangeId exchangeId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - exchId = commState.getDhtPartitionExchangeId("exchId"); + exchId = reader.readMessage("exchId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - lastVer = commState.getCacheVersion("lastVer"); + lastVer = reader.readMessage("lastVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java index fad32399887e7..d8dd3b1a0b2aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java @@ -168,36 +168,36 @@ public void affinityAssignment(List> affAssignment) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 5: - if (!commState.putByteArray("affAssignmentBytes", affAssignmentBytes)) + if (!writer.writeByteArray("affAssignmentBytes", affAssignmentBytes)) return false; - commState.idx++; + state++; case 6: - if (!commState.putByteArray("partsBytes", partsBytes)) + if (!writer.writeByteArray("partsBytes", partsBytes)) return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -207,35 +207,35 @@ public void affinityAssignment(List> affAssignment) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 5: - affAssignmentBytes = commState.getByteArray("affAssignmentBytes"); + affAssignmentBytes = reader.readByteArray("affAssignmentBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - partsBytes = commState.getByteArray("partsBytes"); + partsBytes = reader.readByteArray("partsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java index 94efd43eedd28..d9c9e4527a57c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java @@ -117,24 +117,24 @@ public Map partitions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 5: - if (!commState.putByteArray("partsBytes", partsBytes)) + if (!writer.writeByteArray("partsBytes", partsBytes)) return false; - commState.idx++; + state++; } @@ -144,19 +144,19 @@ public Map partitions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 5: - partsBytes = commState.getByteArray("partsBytes"); + partsBytes = reader.readByteArray("partsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java index d7a091bc628c4..944fcc0ae7f9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleRequest.java @@ -63,16 +63,16 @@ public GridDhtPartitionsSingleRequest() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } return true; @@ -81,7 +81,7 @@ public GridDhtPartitionsSingleRequest() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java index f8c5cc270f683..44088aae6c6e3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java @@ -281,137 +281,84 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putLong("accessTtl", accessTtl)) + if (!writer.writeLong("accessTtl", accessTtl)) return false; - commState.idx++; + state++; case 4: - if (filterBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, filterBytes.length)) - return false; - - commState.it = arrayIterator(filterBytes); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("filterBytes", filterBytes, byte[].class)) + return false; - commState.idx++; + state++; case 5: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 6: - if (keyBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, keyBytes.size())) - return false; - - commState.it = keyBytes.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putByteArray(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putBoolean(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("keyBytes", keyBytes, byte[].class, boolean.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 8: - if (!commState.putBoolean("readThrough", readThrough)) + if (!writer.writeBoolean("readThrough", readThrough)) return false; - commState.idx++; + state++; case 9: - if (!commState.putBoolean("reload", reload)) + if (!writer.writeBoolean("reload", reload)) return false; - commState.idx++; + state++; case 10: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 11: - if (!commState.putCacheVersion("ver", ver)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 12: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 13: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeMessage("ver", ver != null ? ver.clone() : null)) return false; - commState.idx++; + state++; } @@ -421,156 +368,99 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - accessTtl = commState.getLong("accessTtl"); + accessTtl = reader.readLong("accessTtl"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + filterBytes = reader.readObjectArray("filterBytes", byte[].class); - if (commState.readSize >= 0) { - if (filterBytes == null) - filterBytes = new byte[commState.readSize][]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - filterBytes[i] = (byte[])_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (keyBytes == null) - keyBytes = new LinkedHashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - byte[] _val = commState.getByteArray(null); + keyBytes = reader.readMap("keyBytes", byte[].class, boolean.class); - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - boolean _val = commState.getBoolean(null); - - if (!commState.lastRead()) - return false; - - keyBytes.put((byte[])commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - readThrough = commState.getBoolean("readThrough"); + readThrough = reader.readBoolean("readThrough"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - reload = commState.getBoolean("reload"); + reload = reader.readBoolean("reload"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - topVer = commState.getLong("topVer"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - ver = commState.getCacheVersion("ver"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - subjId = commState.getUuid("subjId"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - taskNameHash = commState.getInt("taskNameHash"); + ver = reader.readMessage("ver"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java index cc2a48a82d593..63c9e464bf1cd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetResponse.java @@ -229,81 +229,60 @@ public void error(Throwable err) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putByteArray("entriesBytes", entriesBytes)) + if (!writer.writeByteArray("entriesBytes", entriesBytes)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 6: - if (invalidParts != null) { - if (commState.it == null) { - if (!commState.putInt(null, invalidParts.size())) - return false; - - commState.it = invalidParts.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("invalidParts", invalidParts, int.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 8: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; case 9: - if (!commState.putCacheVersion("ver", ver)) + if (!writer.writeMessage("ver", ver != null ? ver.clone() : null)) return false; - commState.idx++; + state++; } @@ -313,89 +292,67 @@ public void error(Throwable err) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - entriesBytes = commState.getByteArray("entriesBytes"); + entriesBytes = reader.readByteArray("entriesBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + invalidParts = reader.readCollection("invalidParts", int.class); - if (commState.readSize >= 0) { - if (invalidParts == null) - invalidParts = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - invalidParts.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - topVer = commState.getLong("topVer"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - ver = commState.getCacheVersion("ver"); + ver = reader.readMessage("ver"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java index da008ac3522cb..599700760b149 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java @@ -364,132 +364,90 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 24: - if (!commState.putLong("accessTtl", accessTtl)) + if (!writer.writeLong("accessTtl", accessTtl)) return false; - commState.idx++; + state++; case 25: - if (dhtVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, dhtVers.length)) - return false; - - commState.it = arrayIterator(dhtVers); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("dhtVers", dhtVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 26: - if (filterBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, filterBytes.length)) - return false; - - commState.it = arrayIterator(filterBytes); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("filterBytes", filterBytes, byte[].class)) + return false; - commState.idx++; + state++; case 27: - if (!commState.putBoolean("implicitSingleTx", implicitSingleTx)) + if (!writer.writeBoolean("hasTransforms", hasTransforms)) return false; - commState.idx++; + state++; case 28: - if (!commState.putBoolean("implicitTx", implicitTx)) + if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx)) return false; - commState.idx++; + state++; case 29: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeBoolean("implicitTx", implicitTx)) return false; - commState.idx++; + state++; case 30: - if (!commState.putBoolean("onePhaseCommit", onePhaseCommit)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 31: - if (!commState.putBoolean("syncCommit", syncCommit)) + if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit)) return false; - commState.idx++; + state++; case 32: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 33: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeBoolean("syncCommit", syncCommit)) return false; - commState.idx++; + state++; case 34: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 35: - if (!commState.putBoolean("hasTransforms", hasTransforms)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -499,151 +457,107 @@ public long accessTtl() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 24: - accessTtl = commState.getLong("accessTtl"); + accessTtl = reader.readLong("accessTtl"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 25: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + dhtVers = reader.readObjectArray("dhtVers", GridCacheVersion.class); - if (commState.readSize >= 0) { - if (dhtVers == null) - dhtVers = new GridCacheVersion[commState.readSize]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - dhtVers[i] = (GridCacheVersion)_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 26: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + filterBytes = reader.readObjectArray("filterBytes", byte[].class); - if (commState.readSize >= 0) { - if (filterBytes == null) - filterBytes = new byte[commState.readSize][]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - filterBytes[i] = (byte[])_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 27: - implicitSingleTx = commState.getBoolean("implicitSingleTx"); + hasTransforms = reader.readBoolean("hasTransforms"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 28: - implicitTx = commState.getBoolean("implicitTx"); + implicitSingleTx = reader.readBoolean("implicitSingleTx"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 29: - miniId = commState.getGridUuid("miniId"); + implicitTx = reader.readBoolean("implicitTx"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 30: - onePhaseCommit = commState.getBoolean("onePhaseCommit"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 31: - syncCommit = commState.getBoolean("syncCommit"); + onePhaseCommit = reader.readBoolean("onePhaseCommit"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 32: - topVer = commState.getLong("topVer"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 33: - subjId = commState.getUuid("subjId"); + syncCommit = reader.readBoolean("syncCommit"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 34: - taskNameHash = commState.getInt("taskNameHash"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 35: - hasTransforms = commState.getBoolean("hasTransforms"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java index 9a62b025b3edc..2769f7abe3fd1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java @@ -206,111 +206,48 @@ public void addValueBytes( /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 11: - if (dhtVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, dhtVers.length)) - return false; - - commState.it = arrayIterator(dhtVers); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("dhtVers", dhtVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 12: - if (!commState.putBooleanArray("filterRes", filterRes)) + if (!writer.writeBooleanArray("filterRes", filterRes)) return false; - commState.idx++; + state++; case 13: - if (mappedVers != null) { - if (commState.it == null) { - if (!commState.putInt(null, mappedVers.length)) - return false; - - commState.it = arrayIterator(mappedVers); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeObjectArray("mappedVers", mappedVers, GridCacheVersion.class)) + return false; - commState.idx++; + state++; case 14: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 15: - if (pending != null) { - if (commState.it == null) { - if (!commState.putInt(null, pending.size())) - return false; - - commState.it = pending.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("pending", pending, GridCacheVersion.class)) + return false; - commState.idx++; + state++; } @@ -320,117 +257,51 @@ public void addValueBytes( /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 11: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (dhtVers == null) - dhtVers = new GridCacheVersion[commState.readSize]; + dhtVers = reader.readObjectArray("dhtVers", GridCacheVersion.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - dhtVers[i] = (GridCacheVersion)_val; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 12: - filterRes = commState.getBooleanArray("filterRes"); + filterRes = reader.readBooleanArray("filterRes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (mappedVers == null) - mappedVers = new GridCacheVersion[commState.readSize]; - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - mappedVers[i] = (GridCacheVersion)_val; + mappedVers = reader.readObjectArray("mappedVers", GridCacheVersion.class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 14: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 15: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (pending == null) - pending = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); + pending = reader.readCollection("pending", GridCacheVersion.class); - if (!commState.lastRead()) - return false; - - pending.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java index a462751ab25a5..cfa797a0296c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishRequest.java @@ -184,54 +184,54 @@ public int taskNameHash() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 21: - if (!commState.putBoolean("explicitLock", explicitLock)) + if (!writer.writeBoolean("explicitLock", explicitLock)) return false; - commState.idx++; + state++; case 22: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 23: - if (!commState.putBoolean("storeEnabled", storeEnabled)) + if (!writer.writeBoolean("storeEnabled", storeEnabled)) return false; - commState.idx++; + state++; case 24: - if (!commState.putLong("topVer", topVer)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 25: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeInt("taskNameHash", taskNameHash)) return false; - commState.idx++; + state++; case 26: - if (!commState.putInt("taskNameHash", taskNameHash)) + if (!writer.writeLong("topVer", topVer)) return false; - commState.idx++; + state++; } @@ -241,59 +241,59 @@ public int taskNameHash() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 21: - explicitLock = commState.getBoolean("explicitLock"); + explicitLock = reader.readBoolean("explicitLock"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 22: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 23: - storeEnabled = commState.getBoolean("storeEnabled"); + storeEnabled = reader.readBoolean("storeEnabled"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 24: - topVer = commState.getLong("topVer"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 25: - subjId = commState.getUuid("subjId"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 26: - taskNameHash = commState.getInt("taskNameHash"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java index 5532b883ab6b9..5e190792163e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishResponse.java @@ -138,36 +138,36 @@ public long threadId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 5: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 6: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("nearThreadId", nearThreadId)) + if (!writer.writeLong("nearThreadId", nearThreadId)) return false; - commState.idx++; + state++; } @@ -177,35 +177,35 @@ public long threadId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 5: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - nearThreadId = commState.getLong("nearThreadId"); + nearThreadId = reader.readLong("nearThreadId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java index 1cd7ce06d9cf1..9d5f73d89a1c1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java @@ -244,87 +244,66 @@ private Collection> cloneEntries(Collection> cloneEntries(Collection= 0) { - if (lastBackups == null) - lastBackups = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - lastBackups.add((UUID)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 25: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 26: - near = commState.getBoolean("near"); + near = reader.readBoolean("near"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 27: - topVer = commState.getLong("topVer"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 28: - subjId = commState.getUuid("subjId"); + taskNameHash = reader.readInt("taskNameHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 29: - taskNameHash = commState.getInt("taskNameHash"); + topVer = reader.readLong("topVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java index f6e69753f2ab5..8eb63b147550b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java @@ -251,117 +251,54 @@ public Collection invalidPartitions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 10: - if (!commState.putCacheVersion("dhtVer", dhtVer)) + if (!writer.writeMessage("dhtVer", dhtVer != null ? dhtVer.clone() : null)) return false; - commState.idx++; + state++; case 11: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 12: - if (invalidParts != null) { - if (commState.it == null) { - if (!commState.putInt(null, invalidParts.size())) - return false; - - commState.it = invalidParts.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putInt(null, (int)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("invalidParts", invalidParts, int.class)) + return false; - commState.idx++; + state++; case 13: - if (!commState.putGridUuid("miniId", miniId)) + if (!writer.writeIgniteUuid("miniId", miniId)) return false; - commState.idx++; + state++; case 14: - if (ownedValsBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, ownedValsBytes.size())) - return false; - - commState.it = ownedValsBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("ownedValsBytes", ownedValsBytes, byte[].class)) + return false; - commState.idx++; + state++; case 15: - if (pending != null) { - if (commState.it == null) { - if (!commState.putInt(null, pending.size())) - return false; - - commState.it = pending.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putCacheVersion(null, (GridCacheVersion)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("pending", pending, GridCacheVersion.class)) + return false; - commState.idx++; + state++; } @@ -371,125 +308,59 @@ public Collection invalidPartitions() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 10: - dhtVer = commState.getCacheVersion("dhtVer"); + dhtVer = reader.readMessage("dhtVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + invalidParts = reader.readCollection("invalidParts", int.class); - if (commState.readSize >= 0) { - if (invalidParts == null) - invalidParts = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - - invalidParts.add((Integer)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 13: - miniId = commState.getGridUuid("miniId"); + miniId = reader.readIgniteUuid("miniId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (ownedValsBytes == null) - ownedValsBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - ownedValsBytes.add((byte[])_val); + ownedValsBytes = reader.readCollection("ownedValsBytes", byte[].class); - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 15: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (pending == null) - pending = new ArrayList<>(commState.readSize); + pending = reader.readCollection("pending", GridCacheVersion.class); - for (int i = commState.readItems; i < commState.readSize; i++) { - GridCacheVersion _val = commState.getCacheVersion(null); - - if (!commState.lastRead()) - return false; - - pending.add((GridCacheVersion)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java index 86f839bc3cd3e..267d798422d9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearUnlockRequest.java @@ -65,16 +65,16 @@ public GridNearUnlockRequest(int cacheId, int keyCnt) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } return true; @@ -83,7 +83,7 @@ public GridNearUnlockRequest(int cacheId, int keyCnt) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java index 2893f4f6c24f6..e7079d107c170 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java @@ -485,132 +485,132 @@ public int taskHash() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (!commState.putBoolean("all", all)) + if (!writer.writeBoolean("all", all)) return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("argsBytes", argsBytes)) + if (!writer.writeByteArray("argsBytes", argsBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putString("cacheName", cacheName)) + if (!writer.writeString("cacheName", cacheName)) return false; - commState.idx++; + state++; case 6: - if (!commState.putBoolean("cancel", cancel)) + if (!writer.writeBoolean("cancel", cancel)) return false; - commState.idx++; + state++; case 7: - if (!commState.putString("clause", clause)) + if (!writer.writeString("clause", clause)) return false; - commState.idx++; + state++; case 8: - if (!commState.putString("clsName", clsName)) + if (!writer.writeString("clsName", clsName)) return false; - commState.idx++; + state++; case 9: - if (!commState.putBoolean("fields", fields)) + if (!writer.writeBoolean("fields", fields)) return false; - commState.idx++; + state++; case 10: - if (!commState.putLong("id", id)) + if (!writer.writeLong("id", id)) return false; - commState.idx++; + state++; case 11: - if (!commState.putBoolean("incBackups", incBackups)) + if (!writer.writeBoolean("incBackups", incBackups)) return false; - commState.idx++; + state++; case 12: - if (!commState.putBoolean("incMeta", incMeta)) + if (!writer.writeBoolean("incMeta", incMeta)) return false; - commState.idx++; + state++; case 13: - if (!commState.putByteArray("keyValFilterBytes", keyValFilterBytes)) + if (!writer.writeBoolean("keepPortable", keepPortable)) return false; - commState.idx++; + state++; case 14: - if (!commState.putInt("pageSize", pageSize)) + if (!writer.writeByteArray("keyValFilterBytes", keyValFilterBytes)) return false; - commState.idx++; + state++; case 15: - if (!commState.putByteArray("prjFilterBytes", prjFilterBytes)) + if (!writer.writeInt("pageSize", pageSize)) return false; - commState.idx++; + state++; case 16: - if (!commState.putByteArray("rdcBytes", rdcBytes)) + if (!writer.writeByteArray("prjFilterBytes", prjFilterBytes)) return false; - commState.idx++; + state++; case 17: - if (!commState.putByteArray("transBytes", transBytes)) + if (!writer.writeByteArray("rdcBytes", rdcBytes)) return false; - commState.idx++; + state++; case 18: - if (!commState.putEnum("type", type)) + if (!writer.writeUuid("subjId", subjId)) return false; - commState.idx++; + state++; case 19: - if (!commState.putBoolean("keepPortable", keepPortable)) + if (!writer.writeInt("taskHash", taskHash)) return false; - commState.idx++; + state++; case 20: - if (!commState.putUuid("subjId", subjId)) + if (!writer.writeByteArray("transBytes", transBytes)) return false; - commState.idx++; + state++; case 21: - if (!commState.putInt("taskHash", taskHash)) + if (!writer.writeEnum("type", type)) return false; - commState.idx++; + state++; } @@ -620,165 +620,163 @@ public int taskHash() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - all = commState.getBoolean("all"); + all = reader.readBoolean("all"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - argsBytes = commState.getByteArray("argsBytes"); + argsBytes = reader.readByteArray("argsBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - cacheName = commState.getString("cacheName"); + cacheName = reader.readString("cacheName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - cancel = commState.getBoolean("cancel"); + cancel = reader.readBoolean("cancel"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - clause = commState.getString("clause"); + clause = reader.readString("clause"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - clsName = commState.getString("clsName"); + clsName = reader.readString("clsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - fields = commState.getBoolean("fields"); + fields = reader.readBoolean("fields"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - id = commState.getLong("id"); + id = reader.readLong("id"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - incBackups = commState.getBoolean("incBackups"); + incBackups = reader.readBoolean("incBackups"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - incMeta = commState.getBoolean("incMeta"); + incMeta = reader.readBoolean("incMeta"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 13: - keyValFilterBytes = commState.getByteArray("keyValFilterBytes"); + keepPortable = reader.readBoolean("keepPortable"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 14: - pageSize = commState.getInt("pageSize"); + keyValFilterBytes = reader.readByteArray("keyValFilterBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 15: - prjFilterBytes = commState.getByteArray("prjFilterBytes"); + pageSize = reader.readInt("pageSize"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 16: - rdcBytes = commState.getByteArray("rdcBytes"); + prjFilterBytes = reader.readByteArray("prjFilterBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 17: - transBytes = commState.getByteArray("transBytes"); + rdcBytes = reader.readByteArray("rdcBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 18: - byte type0 = commState.getByte("type"); + subjId = reader.readUuid("subjId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - type = GridCacheQueryType.fromOrdinal(type0); - - commState.idx++; + state++; case 19: - keepPortable = commState.getBoolean("keepPortable"); + taskHash = reader.readInt("taskHash"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 20: - subjId = commState.getUuid("subjId"); + transBytes = reader.readByteArray("transBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 21: - taskHash = commState.getInt("taskHash"); + type = reader.readEnum("type", GridCacheQueryType.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java index fe18c8c3a9937..acba6f69e521a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryResponse.java @@ -231,96 +231,54 @@ public boolean fields() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 3: - if (dataBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, dataBytes.size())) - return false; - - commState.it = dataBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("dataBytes", dataBytes, byte[].class)) + return false; - commState.idx++; + state++; case 4: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 5: - if (!commState.putBoolean("fields", fields)) + if (!writer.writeBoolean("fields", fields)) return false; - commState.idx++; + state++; case 6: - if (!commState.putBoolean("finished", finished)) + if (!writer.writeBoolean("finished", finished)) return false; - commState.idx++; + state++; case 7: - if (metaDataBytes != null) { - if (commState.it == null) { - if (!commState.putInt(null, metaDataBytes.size())) - return false; - - commState.it = metaDataBytes.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putByteArray(null, (byte[])commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("metaDataBytes", metaDataBytes, byte[].class)) + return false; - commState.idx++; + state++; case 8: - if (!commState.putLong("reqId", reqId)) + if (!writer.writeLong("reqId", reqId)) return false; - commState.idx++; + state++; } @@ -330,103 +288,59 @@ public boolean fields() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 3: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + dataBytes = reader.readCollection("dataBytes", byte[].class); - if (commState.readSize >= 0) { - if (dataBytes == null) - dataBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - dataBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 4: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - fields = commState.getBoolean("fields"); + fields = reader.readBoolean("fields"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - finished = commState.getBoolean("finished"); + finished = reader.readBoolean("finished"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 7: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (metaDataBytes == null) - metaDataBytes = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; + metaDataBytes = reader.readCollection("metaDataBytes", byte[].class); - metaDataBytes.add((byte[])_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 8: - reqId = commState.getLong("reqId"); + reqId = reader.readLong("reqId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java index 15e84acb1236b..77fdf2dabb180 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersion.java @@ -254,12 +254,85 @@ public IgniteUuid asGridUuid() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeLong("globalTime", globalTime)) + return false; + + state++; + + case 1: + if (!writer.writeInt("nodeOrderDrId", nodeOrderDrId)) + return false; + + state++; + + case 2: + if (!writer.writeLong("order", order)) + return false; + + state++; + + case 3: + if (!writer.writeInt("topVer", topVer)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + globalTime = reader.readLong("globalTime"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + nodeOrderDrId = reader.readInt("nodeOrderDrId"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 2: + order = reader.readLong("order"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 3: + topVer = reader.readInt("topVer"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -269,12 +342,21 @@ public IgniteUuid asGridUuid() { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridCacheVersion _clone = new GridCacheVersion(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridCacheVersion _clone = (GridCacheVersion)_msg; + + _clone.topVer = topVer; + _clone.nodeOrderDrId = nodeOrderDrId; + _clone.globalTime = globalTime; + _clone.order = order; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java index dd576e3f5403c..61fe04480612e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaSnapshotMessage.java @@ -92,59 +92,27 @@ public Map deltas() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (deltas != null) { - if (commState.it == null) { - if (!commState.putInt(null, deltas.size())) - return false; - - commState.it = deltas.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putLong(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("deltas", deltas, UUID.class, long.class)) + return false; - commState.idx++; + state++; case 1: - if (!commState.putClockDeltaVersion("snapVer", snapVer)) + if (!writer.writeMessage("snapVer", snapVer != null ? snapVer.clone() : null)) return false; - commState.idx++; + state++; } @@ -154,59 +122,24 @@ public Map deltas() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + deltas = reader.readMap("deltas", UUID.class, long.class); - if (commState.readSize >= 0) { - if (deltas == null) - deltas = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - long _val = commState.getLong(null); - - if (!commState.lastRead()) - return false; - - deltas.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 1: - snapVer = commState.getClockDeltaVersion("snapVer"); + snapVer = reader.readMessage("snapVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java index 852f9f0eea83b..06e8730fc2289 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockDeltaVersion.java @@ -114,12 +114,57 @@ public long topologyVersion() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeLong("topVer", topVer)) + return false; + + state++; + + case 1: + if (!writer.writeLong("ver", ver)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + topVer = reader.readLong("topVer"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + ver = reader.readLong("ver"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -129,12 +174,19 @@ public long topologyVersion() { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridClockDeltaVersion _clone = new GridClockDeltaVersion(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridClockDeltaVersion _clone = (GridClockDeltaVersion)_msg; + + _clone.ver = ver; + _clone.topVer = topVer; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java index a6a62d750ca42..e005a9b30197b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessage.java @@ -154,39 +154,39 @@ public void dataBytes(byte[] dataBytes) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("dataBytes", dataBytes)) + if (!writer.writeByteArray("dataBytes", dataBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putUuid("routineId", routineId)) + if (!writer.writeUuid("routineId", routineId)) return false; - commState.idx++; + state++; case 3: - if (!commState.putEnum("type", type)) + if (!writer.writeEnum("type", type)) return false; - commState.idx++; + state++; } @@ -196,42 +196,40 @@ public void dataBytes(byte[] dataBytes) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - dataBytes = commState.getByteArray("dataBytes"); + dataBytes = reader.readByteArray("dataBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - routineId = commState.getUuid("routineId"); + routineId = reader.readUuid("routineId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - byte type0 = commState.getByte("type"); + type = reader.readEnum("type", GridContinuousMessageType.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - type = GridContinuousMessageType.fromOrdinal(type0); - - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadRequest.java index 4690f82a8a162..15721f91a81ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadRequest.java @@ -224,125 +224,93 @@ public boolean forceLocalDeployment() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putString("cacheName", cacheName)) + if (!writer.writeString("cacheName", cacheName)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putByteArray("colBytes", colBytes)) + if (!writer.writeByteArray("colBytes", colBytes)) return false; - commState.idx++; + state++; case 3: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 4: - if (!commState.putBoolean("forceLocDep", forceLocDep)) + if (!writer.writeBoolean("forceLocDep", forceLocDep)) return false; - commState.idx++; + state++; case 5: - if (!commState.putBoolean("ignoreDepOwnership", ignoreDepOwnership)) + if (!writer.writeBoolean("ignoreDepOwnership", ignoreDepOwnership)) return false; - commState.idx++; + state++; case 6: - if (ldrParticipants != null) { - if (commState.it == null) { - if (!commState.putInt(null, ldrParticipants.size())) - return false; - - commState.it = ldrParticipants.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("ldrParticipants", ldrParticipants, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 7: - if (!commState.putLong("reqId", reqId)) + if (!writer.writeLong("reqId", reqId)) return false; - commState.idx++; + state++; case 8: - if (!commState.putByteArray("resTopicBytes", resTopicBytes)) + if (!writer.writeByteArray("resTopicBytes", resTopicBytes)) return false; - commState.idx++; + state++; case 9: - if (!commState.putString("sampleClsName", sampleClsName)) + if (!writer.writeString("sampleClsName", sampleClsName)) return false; - commState.idx++; + state++; case 10: - if (!commState.putBoolean("skipStore", skipStore)) + if (!writer.writeBoolean("skipStore", skipStore)) return false; - commState.idx++; + state++; case 11: - if (!commState.putByteArray("updaterBytes", updaterBytes)) + if (!writer.writeByteArray("updaterBytes", updaterBytes)) return false; - commState.idx++; + state++; case 12: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -351,149 +319,112 @@ public boolean forceLocalDeployment() { /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - cacheName = commState.getString("cacheName"); + cacheName = reader.readString("cacheName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - colBytes = commState.getByteArray("colBytes"); + colBytes = reader.readByteArray("colBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 4: - forceLocDep = commState.getBoolean("forceLocDep"); + forceLocDep = reader.readBoolean("forceLocDep"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 5: - ignoreDepOwnership = commState.getBoolean("ignoreDepOwnership"); + ignoreDepOwnership = reader.readBoolean("ignoreDepOwnership"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + ldrParticipants = reader.readMap("ldrParticipants", UUID.class, IgniteUuid.class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (ldrParticipants == null) - ldrParticipants = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - ldrParticipants.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 7: - reqId = commState.getLong("reqId"); + reqId = reader.readLong("reqId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 8: - resTopicBytes = commState.getByteArray("resTopicBytes"); + resTopicBytes = reader.readByteArray("resTopicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 9: - sampleClsName = commState.getString("sampleClsName"); + sampleClsName = reader.readString("sampleClsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 10: - skipStore = commState.getBoolean("skipStore"); + skipStore = reader.readBoolean("skipStore"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 11: - updaterBytes = commState.getByteArray("updaterBytes"); + updaterBytes = reader.readByteArray("updaterBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 12: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadResponse.java index 2b8c0ee9b838d..8676e7e3a83f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoadResponse.java @@ -104,33 +104,33 @@ public boolean forceLocalDeployment() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("forceLocDep", forceLocDep)) + if (!writer.writeBoolean("forceLocDep", forceLocDep)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong("reqId", reqId)) + if (!writer.writeLong("reqId", reqId)) return false; - commState.idx++; + state++; } @@ -140,32 +140,32 @@ public boolean forceLocalDeployment() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - forceLocDep = commState.getBoolean("forceLocDep"); + forceLocDep = reader.readBoolean("forceLocDep"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - reqId = commState.getLong("reqId"); + reqId = reader.readLong("reqId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAckMessage.java index 496a114da24bb..6bbb8e41757ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAckMessage.java @@ -127,36 +127,36 @@ public IgniteCheckedException error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("fileId", fileId)) + if (!writer.writeIgniteUuid("fileId", fileId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong("id", id)) + if (!writer.writeLong("id", id)) return false; - commState.idx++; + state++; } @@ -166,35 +166,35 @@ public IgniteCheckedException error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - fileId = commState.getGridUuid("fileId"); + fileId = reader.readIgniteUuid("fileId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - id = commState.getLong("id"); + id = reader.readLong("id"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlockKey.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlockKey.java index bbcbf4241b59e..6fc3776ad1748 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlockKey.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlockKey.java @@ -181,39 +181,39 @@ public long getBlockId() { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("affKey", affKey)) + if (!writer.writeIgniteUuid("affKey", affKey)) return false; - commState.idx++; + state++; case 1: - if (!commState.putLong("blockId", blockId)) + if (!writer.writeLong("blockId", blockId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putBoolean("evictExclude", evictExclude)) + if (!writer.writeBoolean("evictExclude", evictExclude)) return false; - commState.idx++; + state++; case 3: - if (!commState.putGridUuid("fileId", fileId)) + if (!writer.writeIgniteUuid("fileId", fileId)) return false; - commState.idx++; + state++; } @@ -223,40 +223,40 @@ public long getBlockId() { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - affKey = commState.getGridUuid("affKey"); + affKey = reader.readIgniteUuid("affKey"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - blockId = commState.getLong("blockId"); + blockId = reader.readLong("blockId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - evictExclude = commState.getBoolean("evictExclude"); + evictExclude = reader.readBoolean("evictExclude"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - fileId = commState.getGridUuid("fileId"); + fileId = reader.readIgniteUuid("fileId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlocksMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlocksMessage.java index 632104227ddd5..774006675b101 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlocksMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsBlocksMessage.java @@ -107,68 +107,36 @@ public Map blocks() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (blocks != null) { - if (commState.it == null) { - if (!commState.putInt(null, blocks.size())) - return false; - - commState.it = blocks.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putMessage(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putByteArray(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("blocks", blocks, GridGgfsBlockKey.class, byte[].class)) + return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("fileId", fileId)) + if (!writer.writeIgniteUuid("fileId", fileId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong("id", id)) + if (!writer.writeLong("id", id)) return false; - commState.idx++; + state++; } @@ -178,70 +146,35 @@ public Map blocks() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + blocks = reader.readMap("blocks", GridGgfsBlockKey.class, byte[].class); - if (commState.readSize >= 0) { - if (blocks == null) - blocks = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - GridGgfsBlockKey _val = (GridGgfsBlockKey)commState.getMessage(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - byte[] _val = commState.getByteArray(null); - - if (!commState.lastRead()) - return false; - - blocks.put((GridGgfsBlockKey)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 1: - fileId = commState.getGridUuid("fileId"); + fileId = reader.readIgniteUuid("fileId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - id = commState.getLong("id"); + id = reader.readLong("id"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsCommunicationMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsCommunicationMessage.java index b08d64ae754df..860eb4c1f1ed3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsCommunicationMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsCommunicationMessage.java @@ -54,13 +54,13 @@ public void finishUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } return true; @@ -68,7 +68,7 @@ public void finishUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); return true; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDeleteMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDeleteMessage.java index 6be821a95ca0b..18ae5b53a405a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDeleteMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDeleteMessage.java @@ -130,30 +130,30 @@ public IgniteCheckedException error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("id", id)) + if (!writer.writeIgniteUuid("id", id)) return false; - commState.idx++; + state++; } @@ -163,27 +163,27 @@ public IgniteCheckedException error() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - id = commState.getGridUuid("id"); + id = reader.readIgniteUuid("id"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFileAffinityRange.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFileAffinityRange.java index 04c822e19ea4c..edd9d6a74843b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFileAffinityRange.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFileAffinityRange.java @@ -286,45 +286,45 @@ public boolean regionEqual(GridGgfsFileAffinityRange other) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("affKey", affKey)) + if (!writer.writeIgniteUuid("affKey", affKey)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("done", done)) + if (!writer.writeBoolean("done", done)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong("endOff", endOff)) + if (!writer.writeLong("endOff", endOff)) return false; - commState.idx++; + state++; case 3: - if (!commState.putLong("startOff", startOff)) + if (!writer.writeLong("startOff", startOff)) return false; - commState.idx++; + state++; case 4: - if (!commState.putInt("status", status)) + if (!writer.writeInt("status", status)) return false; - commState.idx++; + state++; } @@ -334,48 +334,48 @@ public boolean regionEqual(GridGgfsFileAffinityRange other) { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - affKey = commState.getGridUuid("affKey"); + affKey = reader.readIgniteUuid("affKey"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - done = commState.getBoolean("done"); + done = reader.readBoolean("done"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - endOff = commState.getLong("endOff"); + endOff = reader.readLong("endOff"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - startOff = commState.getLong("startOff"); + startOff = reader.readLong("startOff"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - status = commState.getInt("status"); + status = reader.readInt("status"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerRequest.java index 3ecac2f3a70cc..978a2d66a8ec9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerRequest.java @@ -101,51 +101,30 @@ public Collection fragmentRanges() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("fileId", fileId)) + if (!writer.writeIgniteUuid("fileId", fileId)) return false; - commState.idx++; + state++; case 1: - if (fragmentRanges != null) { - if (commState.it == null) { - if (!commState.putInt(null, fragmentRanges.size())) - return false; - - commState.it = fragmentRanges.iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - if (!commState.putMessage(null, (GridGgfsFileAffinityRange)commState.cur)) - return false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeCollection("fragmentRanges", fragmentRanges, GridGgfsFileAffinityRange.class)) + return false; - commState.idx++; + state++; } @@ -155,49 +134,27 @@ public Collection fragmentRanges() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - fileId = commState.getGridUuid("fileId"); + fileId = reader.readIgniteUuid("fileId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - if (commState.readSize == -1) { - int _val = commState.getInt(null); - - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } + fragmentRanges = reader.readCollection("fragmentRanges", GridGgfsFileAffinityRange.class); - if (commState.readSize >= 0) { - if (fragmentRanges == null) - fragmentRanges = new ArrayList<>(commState.readSize); - - for (int i = commState.readItems; i < commState.readSize; i++) { - GridGgfsFileAffinityRange _val = (GridGgfsFileAffinityRange)commState.getMessage(null); - - if (!commState.lastRead()) - return false; - - fragmentRanges.add((GridGgfsFileAffinityRange)_val); - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerResponse.java index a842e9c98c9da..5a90b3c7c60d0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsFragmentizerResponse.java @@ -76,24 +76,24 @@ public IgniteUuid fileId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("fileId", fileId)) + if (!writer.writeIgniteUuid("fileId", fileId)) return false; - commState.idx++; + state++; } @@ -103,19 +103,19 @@ public IgniteUuid fileId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - fileId = commState.getGridUuid("fileId"); + fileId = reader.readIgniteUuid("fileId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsSyncMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsSyncMessage.java index a632e02363681..c7c1e77053684 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsSyncMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsSyncMessage.java @@ -94,30 +94,30 @@ public boolean response() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); if (!super.writeTo(buf)) return false; - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putLong("order", order)) + if (!writer.writeLong("order", order)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("res", res)) + if (!writer.writeBoolean("res", res)) return false; - commState.idx++; + state++; } @@ -127,27 +127,27 @@ public boolean response() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); if (!super.readFrom(buf)) return false; - switch (commState.idx) { + switch (state) { case 0: - order = commState.getLong("order"); + order = reader.readLong("order"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - res = commState.getBoolean("res"); + res = reader.readBoolean("res"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java index 6223639f1e873..71dfcf94e255c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeRequestWrapper.java @@ -65,13 +65,13 @@ public byte[] bytes() { @Override public boolean writeTo(ByteBuffer buf) { stream.setBuffer(buf); - if (!commState.typeWritten) { + if (!typeWritten) { if (!buf.hasRemaining()) return false; stream.writeByte(directType()); - commState.typeWritten = true; + typeWritten = true; } stream.writeByteArray(bytes, 0, bytes.length); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java index 57f2bd163a752..e598f66716086 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientHandshakeResponseWrapper.java @@ -48,13 +48,13 @@ public GridClientHandshakeResponseWrapper(byte code) { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } return true; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientMessageWrapper.java index 4d40c3ed8ae1f..746a40f5c8203 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientMessageWrapper.java @@ -136,23 +136,23 @@ public void message(ByteBuffer msg) { @Override public boolean writeTo(ByteBuffer buf) { stream.setBuffer(buf); - if (!commState.typeWritten) { + if (!typeWritten) { if (stream.remaining() < 1) return false; stream.writeByte(directType()); - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: if (stream.remaining() < 4) return false; stream.writeInt(msgSize); - commState.idx++; + state++; case 1: if (stream.remaining() < 8) @@ -160,7 +160,7 @@ public void message(ByteBuffer msg) { stream.writeLong(reqId); - commState.idx++; + state++; case 2: if (stream.remaining() < 16) @@ -168,7 +168,7 @@ public void message(ByteBuffer msg) { stream.writeByteArray(U.uuidToBytes(clientId), 0, 16); - commState.idx++; + state++; case 3: if (stream.remaining() < 16) @@ -176,12 +176,12 @@ public void message(ByteBuffer msg) { stream.writeByteArray(U.uuidToBytes(destId), 0, 16); - commState.idx++; + state++; case 4: stream.writeByteArray(msg.array(), msg.position(), msg.remaining()); - commState.idx++; + state++; } @@ -192,7 +192,7 @@ public void message(ByteBuffer msg) { @Override public boolean readFrom(ByteBuffer buf) { stream.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: if (stream.remaining() < 4) return false; @@ -202,7 +202,7 @@ public void message(ByteBuffer msg) { if (msgSize == 0) // Ping message. return true; - commState.idx++; + state++; case 1: if (stream.remaining() < 8) @@ -210,7 +210,7 @@ public void message(ByteBuffer msg) { reqId = stream.readLong(); - commState.idx++; + state++; case 2: if (stream.remaining() < 16) @@ -218,7 +218,7 @@ public void message(ByteBuffer msg) { clientId = U.bytesToUuid(stream.readByteArray(16), 0); - commState.idx++; + state++; case 3: if (stream.remaining() < 16) @@ -226,7 +226,7 @@ public void message(ByteBuffer msg) { destId = U.bytesToUuid(stream.readByteArray(16), 0); - commState.idx++; + state++; case 4: byte[] msg0 = stream.readByteArray(msgSize); @@ -236,7 +236,7 @@ public void message(ByteBuffer msg) { msg = ByteBuffer.wrap(msg0); - commState.idx++; + state++; } return true; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPingPacketWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPingPacketWrapper.java index be953672259e8..ca8e80483b97a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPingPacketWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/client/message/GridClientPingPacketWrapper.java @@ -34,21 +34,21 @@ public class GridClientPingPacketWrapper extends MessageAdapter { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putInt("size", size)) + if (!writer.writeInt("size", size)) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java index 8df3008fd3ccd..637fa805e8638 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultRequest.java @@ -120,27 +120,27 @@ public void topic(String topic) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("taskId", taskId)) + if (!writer.writeIgniteUuid("taskId", taskId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putByteArray("topicBytes", topicBytes)) + if (!writer.writeByteArray("topicBytes", topicBytes)) return false; - commState.idx++; + state++; } @@ -150,24 +150,24 @@ public void topic(String topic) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - taskId = commState.getGridUuid("taskId"); + taskId = reader.readIgniteUuid("taskId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - topicBytes = commState.getByteArray("topicBytes"); + topicBytes = reader.readByteArray("topicBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java index 6808f1adeb6be..384cfb3c7e082 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskResultResponse.java @@ -140,39 +140,39 @@ public void error(String err) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putString("err", err)) + if (!writer.writeString("err", err)) return false; - commState.idx++; + state++; case 1: - if (!commState.putBoolean("finished", finished)) + if (!writer.writeBoolean("finished", finished)) return false; - commState.idx++; + state++; case 2: - if (!commState.putBoolean("found", found)) + if (!writer.writeBoolean("found", found)) return false; - commState.idx++; + state++; case 3: - if (!commState.putByteArray("resBytes", resBytes)) + if (!writer.writeByteArray("resBytes", resBytes)) return false; - commState.idx++; + state++; } @@ -182,40 +182,40 @@ public void error(String err) { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - err = commState.getString("err"); + err = reader.readString("err"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - finished = commState.getBoolean("finished"); + finished = reader.readBoolean("finished"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - found = commState.getBoolean("found"); + found = reader.readBoolean("found"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 3: - resBytes = commState.getByteArray("resBytes"); + resBytes = reader.readByteArray("resBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java index ae0d6385528dc..d086efe812a47 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java @@ -69,13 +69,13 @@ public GridMemcachedMessageWrapper(GridMemcachedMessage msg, Marshaller jdkMarsh @Override public boolean writeTo(ByteBuffer buf) { stream.setBuffer(buf); - if (!commState.typeWritten) { + if (!typeWritten) { if (!buf.hasRemaining()) return false; stream.writeByte(directType()); - commState.typeWritten = true; + typeWritten = true; } stream.writeByteArray(bytes, 0, bytes.length); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerCancelRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerCancelRequest.java index aba17f31c6ef2..4f67c3abee23d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerCancelRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerCancelRequest.java @@ -74,21 +74,21 @@ public IgniteUuid cancelledFutureId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putGridUuid("cancelledFutId", cancelledFutId)) + if (!writer.writeIgniteUuid("cancelledFutId", cancelledFutId)) return false; - commState.idx++; + state++; } @@ -98,16 +98,16 @@ public IgniteUuid cancelledFutureId() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - cancelledFutId = commState.getGridUuid("cancelledFutId"); + cancelledFutId = reader.readIgniteUuid("cancelledFutId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerExecutionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerExecutionRequest.java index e10ef047d0478..90e7163128fc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerExecutionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerExecutionRequest.java @@ -175,89 +175,57 @@ public byte[] batchBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("batchBytes", batchBytes)) + if (!writer.writeByteArray("batchBytes", batchBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("clsLdrId", clsLdrId)) + if (!writer.writeIgniteUuid("clsLdrId", clsLdrId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putEnum("depMode", depMode)) + if (!writer.writeEnum("depMode", depMode)) return false; - commState.idx++; + state++; case 3: - if (!commState.putBoolean("forceLocDep", forceLocDep)) + if (!writer.writeBoolean("forceLocDep", forceLocDep)) return false; - commState.idx++; + state++; case 4: - if (ldrParticipants != null) { - if (commState.it == null) { - if (!commState.putInt(null, ldrParticipants.size())) - return false; - - commState.it = ldrParticipants.entrySet().iterator(); - } - - while (commState.it.hasNext() || commState.cur != NULL) { - if (commState.cur == NULL) - commState.cur = commState.it.next(); - - Map.Entry e = (Map.Entry)commState.cur; - - if (!commState.keyDone) { - if (!commState.putUuid(null, e.getKey())) - return false; - - commState.keyDone = true; - } - - if (!commState.putGridUuid(null, e.getValue())) - return false; - - commState.keyDone = false; - - commState.cur = NULL; - } - - commState.it = null; - } else { - if (!commState.putInt(null, -1)) - return false; - } + if (!writer.writeMap("ldrParticipants", ldrParticipants, UUID.class, IgniteUuid.class)) + return false; - commState.idx++; + state++; case 5: - if (!commState.putString("sampleClsName", sampleClsName)) + if (!writer.writeString("sampleClsName", sampleClsName)) return false; - commState.idx++; + state++; case 6: - if (!commState.putString("userVer", userVer)) + if (!writer.writeString("userVer", userVer)) return false; - commState.idx++; + state++; } @@ -267,101 +235,64 @@ public byte[] batchBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - batchBytes = commState.getByteArray("batchBytes"); + batchBytes = reader.readByteArray("batchBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - clsLdrId = commState.getGridUuid("clsLdrId"); + clsLdrId = reader.readIgniteUuid("clsLdrId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 2: - byte depMode0 = commState.getByte("depMode"); + depMode = reader.readEnum("depMode", DeploymentMode.class); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - depMode = DeploymentMode.fromOrdinal(depMode0); - - commState.idx++; + state++; case 3: - forceLocDep = commState.getBoolean("forceLocDep"); + forceLocDep = reader.readBoolean("forceLocDep"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 4: - if (commState.readSize == -1) { - int _val = commState.getInt(null); + ldrParticipants = reader.readMap("ldrParticipants", UUID.class, IgniteUuid.class); - if (!commState.lastRead()) - return false; - commState.readSize = _val; - } - - if (commState.readSize >= 0) { - if (ldrParticipants == null) - ldrParticipants = new HashMap<>(commState.readSize, 1.0f); - - for (int i = commState.readItems; i < commState.readSize; i++) { - if (!commState.keyDone) { - UUID _val = commState.getUuid(null); - - if (!commState.lastRead()) - return false; - - commState.cur = _val; - commState.keyDone = true; - } - - IgniteUuid _val = commState.getGridUuid(null); - - if (!commState.lastRead()) - return false; - - ldrParticipants.put((UUID)commState.cur, _val); - - commState.keyDone = false; - - commState.readItems++; - } - } - - commState.readSize = -1; - commState.readItems = 0; - commState.cur = null; + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 5: - sampleClsName = commState.getString("sampleClsName"); + sampleClsName = reader.readString("sampleClsName"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 6: - userVer = commState.getString("userVer"); + userVer = reader.readString("userVer"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerResponse.java index a9c05c55ba571..fc6da7fa8f74d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/GridStreamerResponse.java @@ -95,27 +95,27 @@ public byte[] errorBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putByteArray("errBytes", errBytes)) + if (!writer.writeByteArray("errBytes", errBytes)) return false; - commState.idx++; + state++; case 1: - if (!commState.putGridUuid("futId", futId)) + if (!writer.writeIgniteUuid("futId", futId)) return false; - commState.idx++; + state++; } @@ -125,24 +125,24 @@ public byte[] errorBytes() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - errBytes = commState.getByteArray("errBytes"); + errBytes = reader.readByteArray("errBytes"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - futId = commState.getGridUuid("futId"); + futId = reader.readIgniteUuid("futId"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/version/GridVersionConverter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/version/GridVersionConverter.java deleted file mode 100644 index 9895f431eb5e6..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/version/GridVersionConverter.java +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.ignite.internal.processors.version; - -import org.apache.ignite.internal.direct.*; - -import java.nio.*; - -/** - * Version converter. - */ -public abstract class GridVersionConverter { - /** State. */ - protected final GridTcpCommunicationMessageState commState = new GridTcpCommunicationMessageState(); - - /** - * Writes delta between two versions. - * - * @param buf Buffer to write to. - * @return Whether delta was fully written. - */ - public abstract boolean writeTo(ByteBuffer buf); - - /** - * Reads delta between two versions. - * - * @param buf Buffer to read from. - * @return Whether delta was fully read. - */ - public abstract boolean readFrom(ByteBuffer buf); -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java index 9cff49ca0a41e..8ec78d5bae501 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridByteArrayList.java @@ -408,12 +408,57 @@ public InputStream inputStream() { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeByteArray("data", data)) + return false; + + state++; + + case 1: + if (!writer.writeInt("size", size)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + data = reader.readByteArray("data"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + size = reader.readInt("size"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -423,12 +468,19 @@ public InputStream inputStream() { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridByteArrayList _clone = new GridByteArrayList(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridByteArrayList _clone = (GridByteArrayList)_msg; + + _clone.data = data; + _clone.size = size; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java index 25254676ed32c..5dce4e9db22f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLongList.java @@ -506,12 +506,57 @@ public void pop(int cnt) { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - return false; // TODO: implement. + writer.setBuffer(buf); + + if (!typeWritten) { + if (!writer.writeByte(null, directType())) + return false; + + typeWritten = true; + } + + switch (state) { + case 0: + if (!writer.writeLongArray("arr", arr)) + return false; + + state++; + + case 1: + if (!writer.writeInt("idx", idx)) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @Override public boolean readFrom(ByteBuffer buf) { - return false; // TODO: implement. + reader.setBuffer(buf); + + switch (state) { + case 0: + arr = reader.readLongArray("arr"); + + if (!reader.isLastRead()) + return false; + + state++; + + case 1: + idx = reader.readInt("idx"); + + if (!reader.isLastRead()) + return false; + + state++; + + } + + return true; } /** {@inheritDoc} */ @@ -521,11 +566,18 @@ public void pop(int cnt) { /** {@inheritDoc} */ @Override public MessageAdapter clone() { - return null; // TODO: implement. + GridLongList _clone = new GridLongList(); + + clone0(_clone); + + return _clone; } /** {@inheritDoc} */ @Override protected void clone0(MessageAdapter _msg) { - // TODO: implement. + GridLongList _clone = (GridLongList)_msg; + + _clone.arr = arr; + _clone.idx = idx; } } diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageAdapter.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageAdapter.java index 2234698b4a75e..d393e1663c3ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageAdapter.java @@ -17,24 +17,13 @@ package org.apache.ignite.plugin.extensions.communication; -import org.apache.ignite.internal.direct.*; - import java.io.*; import java.nio.*; -import java.util.*; /** * Communication message adapter. */ public abstract class MessageAdapter implements Serializable, Cloneable { - /** */ - // TODO: remove - protected static final Object NULL = new Object(); - - /** */ - // TODO: remove - protected final GridTcpCommunicationMessageState commState = new GridTcpCommunicationMessageState(); - /** Writer. */ protected MessageWriter writer; @@ -63,17 +52,6 @@ public final void setReader(MessageReader reader) { this.reader = reader; } - /** - * @param buf Buffer. - */ - public final void setBuffer(ByteBuffer buf) { - if (writer != null) - writer.setBuffer(buf); - - if (reader != null) - reader.setBuffer(buf); - } - /** * @param buf Byte buffer. * @return Whether message was fully written. @@ -108,29 +86,4 @@ public final void setBuffer(ByteBuffer buf) { public boolean skipRecovery() { return false; } - - /** - * @param arr Array. - * @return Array iterator. - */ - protected final Iterator arrayIterator(final Object[] arr) { - return new Iterator() { - private int idx; - - @Override public boolean hasNext() { - return idx < arr.length; - } - - @Override public Object next() { - if (!hasNext()) - throw new NoSuchElementException(); - - return arr[idx++]; - } - - @Override public void remove() { - throw new UnsupportedOperationException(); - } - }; - } } diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java index eca6afbca06e8..db4d856eae322 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageReader.java @@ -68,7 +68,7 @@ public interface MessageReader { public , T> C readCollection(String name, Class itemCls); - public Map readMap(String name, Class keyCls, Class valCls); + public , K, V> M readMap(String name, Class keyCls, Class valCls); public boolean isLastRead(); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java index d468c281a8736..9a3a0f0d3c2cf 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/collision/jobstealing/JobStealingRequest.java @@ -74,21 +74,21 @@ int delta() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putInt("delta", delta)) + if (!writer.writeInt("delta", delta)) return false; - commState.idx++; + state++; } @@ -98,16 +98,16 @@ int delta() { /** {@inheritDoc} */ @SuppressWarnings("all") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - delta = commState.getInt("delta"); + delta = reader.readInt("delta"); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java index ce1d7654cec5c..d99cdfdb89366 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java @@ -156,9 +156,9 @@ private static class TestMessage extends MessageAdapter { /** {@inheritDoc} */ @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - return commState.putByte(null, directType()); + return writer.writeByte(null, directType()); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java index e1244a0b69a0c..9089e7c454acc 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java @@ -116,39 +116,39 @@ public byte[] payload() { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean writeTo(ByteBuffer buf) { - commState.setBuffer(buf); + writer.setBuffer(buf); - if (!commState.typeWritten) { - if (!commState.putByte(null, directType())) + if (!typeWritten) { + if (!writer.writeByte(null, directType())) return false; - commState.typeWritten = true; + typeWritten = true; } - switch (commState.idx) { + switch (state) { case 0: - if (!commState.putUuid(null, srcNodeId)) + if (!writer.writeUuid(null, srcNodeId)) return false; - commState.idx++; + state++; case 1: - if (!commState.putLong(null, msgId)) + if (!writer.writeLong(null, msgId)) return false; - commState.idx++; + state++; case 2: - if (!commState.putLong(null, resId)) + if (!writer.writeLong(null, resId)) return false; - commState.idx++; + state++; case 3: - if (!commState.putByteArray(null, payload)) + if (!writer.writeByteArray(null, payload)) return false; - commState.idx++; + state++; } return true; @@ -157,40 +157,40 @@ public byte[] payload() { /** {@inheritDoc} */ @SuppressWarnings("fallthrough") @Override public boolean readFrom(ByteBuffer buf) { - commState.setBuffer(buf); + reader.setBuffer(buf); - switch (commState.idx) { + switch (state) { case 0: - srcNodeId = commState.getUuid(null); + srcNodeId = reader.readUuid(null); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; case 1: - if (buf.remaining() < 8) - return false; + msgId = reader.readLong(null); - msgId = commState.getLong(null); + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 2: - if (buf.remaining() < 8) - return false; + resId = reader.readLong(null); - resId = commState.getLong(null); + if (!reader.isLastRead()) + return false; - commState.idx++; + state++; case 3: - payload = commState.getByteArray(null); + payload = reader.readByteArray(null); - if (!commState.lastRead()) + if (!reader.isLastRead()) return false; - commState.idx++; + state++; } return true;