From 85968d8afb1687fcfb309202fa9da3822e789f58 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Tue, 7 Apr 2026 17:24:19 +0300 Subject: [PATCH 1/9] impl --- .../apache/ignite/internal/ExchangeInfo.java | 4 +- .../ignite/internal/GridKernalContext.java | 8 + .../internal/GridKernalContextImpl.java | 24 +- .../internal/IgniteDiagnosticRequest.java | 4 +- .../internal/IgniteDiagnosticResponse.java | 4 +- .../apache/ignite/internal/IgniteKernal.java | 40 + .../apache/ignite/internal/TxEntriesInfo.java | 4 +- .../org/apache/ignite/internal/TxInfo.java | 4 +- .../managers/CoreMessagesProvider.java | 681 ++++++++++++++++++ .../managers/communication/GridIoManager.java | 26 +- .../communication/GridIoMessageFactory.java | 581 --------------- .../discovery/DiscoveryMessageFactory.java | 371 ---------- .../SecurityAwareCustomMessageWrapper.java | 4 +- .../ChangeCacheEncryptionRequest.java | 4 +- .../encryption/MasterKeyChangeRequest.java | 4 +- .../CacheStatisticsModeChangeMessage.java | 4 +- .../cache/ExchangeFailureMessage.java | 4 +- .../GridDistributedTxFinishResponse.java | 4 +- .../AbstractSnapshotOperationRequest.java | 4 +- .../DataStreamerUpdatesHandlerResult.java | 5 +- .../IncrementalSnapshotVerifyResult.java | 4 +- .../SnapshotCheckHandlersResponse.java | 4 +- .../SnapshotCheckPartitionHashesResponse.java | 4 +- .../snapshot/SnapshotCheckProcessRequest.java | 4 +- .../snapshot/SnapshotCheckResponse.java | 4 +- .../snapshot/SnapshotHandlerResult.java | 4 +- .../snapshot/SnapshotMetadataResponse.java | 4 +- .../snapshot/SnapshotOperationEndRequest.java | 4 +- .../snapshot/SnapshotOperationRequest.java | 4 +- .../snapshot/SnapshotOperationResponse.java | 4 +- ...apshotPartitionsVerifyHandlerResponse.java | 4 +- .../SnapshotRestoreOperationResponse.java | 4 +- .../snapshot/SnapshotRestoreStartRequest.java | 4 +- .../SnapshotStartDiscoveryMessage.java | 4 +- .../reader/StandaloneGridKernalContext.java | 6 + .../cluster/CacheMetricsMessage.java | 4 +- .../cluster/ClusterMetricsUpdateMessage.java | 4 +- .../cluster/NodeFullMetricsMessage.java | 4 +- .../DistributedMetaStorageCasAckMessage.java | 4 +- .../DistributedMetaStorageCasMessage.java | 4 +- ...istributedMetaStorageUpdateAckMessage.java | 4 +- .../DistributedMetaStorageUpdateMessage.java | 4 +- .../service/ServiceChangeBatchRequest.java | 4 +- .../ServiceClusterDeploymentResult.java | 4 +- .../ServiceClusterDeploymentResultBatch.java | 4 +- .../service/ServiceDeploymentRequest.java | 4 +- .../service/ServiceUndeploymentRequest.java | 4 +- .../util/distributed/FullMessage.java | 4 +- .../util/distributed/InitMessage.java | 4 +- .../util/distributed/SingleNodeMessage.java | 4 +- .../ignite/lang/IgniteProductVersion.java | 9 +- .../extensions/communication/Message.java | 3 +- .../communication/MessageFactoryProvider.java | 5 +- .../tcp/TcpCommunicationSpi.java | 12 - .../tcp/internal/GridNioServerWrapper.java | 4 +- .../TcpConnectionRequestDiscoveryMessage.java | 4 +- .../tcp/internal/TcpHandshakeExecutor.java | 7 +- .../spi/discovery/tcp/TcpDiscoveryImpl.java | 4 +- .../spi/discovery/tcp/TcpDiscoverySpi.java | 8 +- .../tcp/internal/TcpDiscoveryNode.java | 11 +- .../tcp/messages/InetAddressMessage.java | 4 +- .../messages/InetSocketAddressMessage.java | 4 +- .../TcpDiscoveryAbstractTraceableMessage.java | 4 +- .../TcpDiscoveryAuthFailedMessage.java | 4 +- .../TcpDiscoveryCacheMetricsMessage.java | 46 -- ...cpDiscoveryClientMetricsUpdateMessage.java | 11 +- ...TcpDiscoveryClientNodesMetricsMessage.java | 11 +- .../TcpDiscoveryClientReconnectMessage.java | 4 +- .../TcpDiscoveryCollectionMessage.java | 4 +- .../TcpDiscoveryConnectionCheckMessage.java | 4 +- .../TcpDiscoveryCustomEventMessage.java | 4 +- .../TcpDiscoveryHandshakeRequest.java | 4 +- .../TcpDiscoveryHandshakeResponse.java | 4 +- .../TcpDiscoveryMetricsUpdateMessage.java | 20 +- .../TcpDiscoveryNodeAddedMessage.java | 9 +- .../TcpDiscoveryNodeFullMetricsMessage.java | 40 - .../messages/TcpDiscoveryNodeLeftMessage.java | 4 +- .../TcpDiscoveryNodeMetricsMessage.java | 46 -- .../TcpDiscoveryRingLatencyCheckMessage.java | 4 +- .../TcpDiscoveryStatusCheckMessage.java | 4 +- .../direct/DirectMarshallingMessagesTest.java | 4 +- .../direct/TestNestedContainersMessage.java | 4 +- .../communication/CompressedMessageTest.java | 3 +- ...CommunicationMessageSerializationTest.java | 3 +- .../IgniteMessageFactoryImplTest.java | 3 +- .../MessageDirectTypeIdConflictTest.java | 6 +- ...niteDiscoveryMessageSerializationTest.java | 3 +- ...acheContinuousQueryImmutableEntryTest.java | 4 +- .../distributed/MessagesPluginProvider.java | 4 +- .../util/distributed/TestIntegerMessage.java | 4 +- .../util/distributed/TestUuidMessage.java | 4 +- .../GridAbstractCommunicationSelfTest.java | 4 +- ...unicationSpiConcurrentConnectSelfTest.java | 4 +- ...GridTcpCommunicationSpiConfigSelfTest.java | 4 +- ...CommunicationSpiMultithreadedSelfTest.java | 4 +- ...cpCommunicationSpiRecoveryAckSelfTest.java | 4 +- ...idTcpCommunicationSpiRecoverySelfTest.java | 4 +- ...ationSpiSkipWaitHandshakeOnClientTest.java | 4 +- ...mmunicationRecoveryAckClosureSelfTest.java | 4 +- .../testframework/GridSpiTestContext.java | 4 +- .../zk/internal/DiscoveryMessageParser.java | 5 +- 101 files changed, 943 insertions(+), 1348 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCacheMetricsMessage.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFullMetricsMessage.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeMetricsMessage.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java index 62c0a0fce7a36..5fc106e390754 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java @@ -19,7 +19,7 @@ import java.util.List; import java.util.Objects; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.util.typedef.internal.U; @@ -31,7 +31,7 @@ public final class ExchangeInfo extends IgniteDiagnosticRequest.DiagnosticBaseIn AffinityTopologyVersion topVer; /** - * Empty constructor required by {@link GridIoMessageFactory}. + * Empty constructor required by {@link CoreMessagesProvider}. */ public ExchangeInfo() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java index 742105bf4da59..c46f28b0eda4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java @@ -84,6 +84,7 @@ import org.apache.ignite.maintenance.MaintenanceRegistry; import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * @@ -209,6 +210,13 @@ public interface GridKernalContext extends Iterable { */ public MaintenanceRegistry maintenanceRegistry(); + /** + * Gets core message factoy. + * + * @return Core message factory. + */ + public MessageFactory messageFactory(); + /** * Gets transformation processor. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java index 2a7c7f66afa67..e1676be483ebf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java @@ -114,6 +114,7 @@ import org.apache.ignite.maintenance.MaintenanceRegistry; import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.IgniteComponentType.SPRING; @@ -371,7 +372,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable private Thread.UncaughtExceptionHandler hnd; /** */ - private IgniteEx grid; + private IgniteKernal grid; /** */ private IgniteConfiguration cfg; @@ -379,9 +380,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable /** */ private GridKernalGateway gw; - /** Network segmented flag. */ - private volatile boolean segFlag; - /** Performance suggestions. */ private final GridPerformanceSuggestions perf = new GridPerformanceSuggestions(); @@ -430,7 +428,7 @@ public GridKernalContextImpl() { @SuppressWarnings("TypeMayBeWeakened") protected GridKernalContextImpl( GridLoggerProxy log, - IgniteEx grid, + IgniteKernal grid, IgniteConfiguration cfg, GridKernalGateway gw, List plugins, @@ -614,18 +612,9 @@ else if (!(comp instanceof DiscoveryNodeValidationProcessor comps.add(comp); } - /** - * @param helper Helper to add. - */ - public void addHelper(Object helper) { - assert helper != null; - - assert false : "Unknown helper class: " + helper.getClass(); - } - /** {@inheritDoc} */ @Override public boolean isStopping() { - return ((IgniteKernal)grid).isStopping(); + return grid.isStopping(); } /** */ @@ -703,6 +692,11 @@ public void addHelper(Object helper) { return maintenanceProc; } + /** {@inheritDoc} */ + @Override public MessageFactory messageFactory() { + return grid.messageFactory(); + } + /** {@inheritDoc} */ @Override public CacheObjectTransformerProcessor transformer() { return transProc; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java index 18276f939b49a..b0a67c4bf02e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java @@ -24,7 +24,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -49,7 +49,7 @@ public class IgniteDiagnosticRequest implements Message { private final Map> msgs = new LinkedHashMap<>(); /** - * Default constructor required by {@link GridIoMessageFactory}. + * Default constructor required by {@link CoreMessagesProvider}. */ public IgniteDiagnosticRequest() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java index 3c2aac181e4f3..eaca57f88101b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -33,7 +33,7 @@ public class IgniteDiagnosticResponse implements Message { @Nullable String respInfo; /** - * Default constructor required by {@link GridIoMessageFactory}. + * Default constructor required by {@link CoreMessagesProvider}. */ public IgniteDiagnosticResponse() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index ebaa9d0bbabcd..f81db699f8b02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -94,11 +94,13 @@ import org.apache.ignite.internal.cluster.IgniteClusterEx; import org.apache.ignite.internal.maintenance.MaintenanceProcessor; import org.apache.ignite.internal.management.IgniteCommandRegistry; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.IgniteMBeansManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; import org.apache.ignite.internal.managers.collision.GridCollisionManager; import org.apache.ignite.internal.managers.communication.GridIoManager; +import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.managers.deployment.GridDeploymentManager; import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -209,6 +211,8 @@ import org.apache.ignite.plugin.IgnitePlugin; import org.apache.ignite.plugin.PluginNotFoundException; import org.apache.ignite.plugin.PluginProvider; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; +import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; import org.apache.ignite.spi.IgniteSpi; import org.apache.ignite.spi.IgniteSpiVersionCheckException; import org.apache.ignite.spi.discovery.isolated.IsolatedDiscoverySpi; @@ -436,6 +440,9 @@ public class IgniteKernal implements IgniteEx, Externalizable { /** The state object is used when reconnection occurs. See {@link IgniteKernal#onReconnected(boolean)}. */ private final ReconnectState reconnectState = new ReconnectState(); + /** Core message factory. */ + private MessageFactory msgFactory; + /** * No-arg constructor is required by externalization. */ @@ -999,7 +1006,11 @@ public void start( } startManager(new GridMetricManager(ctx)); startManager(new GridSystemViewManager(ctx)); + + initMessageFactory(); + startManager(new GridIoManager(ctx)); + startManager(new GridCheckpointManager(ctx)); startManager(new GridEventStorageManager(ctx)); @@ -1301,6 +1312,30 @@ else if (e instanceof IgniteCheckedException) startTimer.finishGlobalStage("Await exchange"); } + /** */ + private void initMessageFactory() throws IgniteCheckedException { + MessageFactoryProvider[] msgs = ctx.plugins().extensions(MessageFactoryProvider.class); + + if (msgs == null) + msgs = new MessageFactoryProvider[0]; + + List compMsgs = new ArrayList<>(); + + compMsgs.add(new CoreMessagesProvider(ctx.marshaller(), U.resolveClassLoader(ctx.config()))); + + for (IgniteComponentType compType : IgniteComponentType.values()) { + MessageFactoryProvider f = compType.messageFactory(); + + if (f != null) + compMsgs.add(f); + } + + if (!compMsgs.isEmpty()) + msgs = F.concat(msgs, compMsgs.toArray(new MessageFactoryProvider[compMsgs.size()])); + + msgFactory = new IgniteMessageFactoryImpl(msgs); + } + /** * @return Ignite security processor. See {@link IgniteSecurity} for details. */ @@ -3028,6 +3063,11 @@ private void checkClusterState() throws IgniteException { } } + /** @return Core message factory. */ + MessageFactory messageFactory() { + return msgFactory; + } + /** * Method is responsible for handling the {@link EventType#EVT_CLIENT_NODE_DISCONNECTED} event. Notify all the * GridComponents that the such even has been occurred (e.g. if the local client node disconnected from the cluster diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java index 150c8719d306a..b14a39411815a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java @@ -21,7 +21,7 @@ import java.util.HashSet; import java.util.Objects; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.KeyCacheObject; @@ -38,7 +38,7 @@ public final class TxEntriesInfo extends IgniteDiagnosticRequest.DiagnosticBaseI Collection keys; /** - * Empty constructor required by {@link GridIoMessageFactory}. + * Empty constructor required by {@link CoreMessagesProvider}. */ public TxEntriesInfo() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java index 9158ead811c45..89ab35a2d0c72 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java @@ -18,7 +18,7 @@ package org.apache.ignite.internal; import java.util.Objects; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.internal.U; @@ -34,7 +34,7 @@ public final class TxInfo extends IgniteDiagnosticRequest.DiagnosticBaseInfo { GridCacheVersion nearVer; /** - * Empty constructor required by {@link GridIoMessageFactory}. + * Empty constructor required by {@link CoreMessagesProvider}. */ public TxInfo() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java new file mode 100644 index 0000000000000..49f35b6ade371 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java @@ -0,0 +1,681 @@ +/* + * 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.managers; + +import java.lang.reflect.Constructor; +import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.ExchangeInfo; +import org.apache.ignite.internal.GridJobCancelRequest; +import org.apache.ignite.internal.GridJobExecuteRequest; +import org.apache.ignite.internal.GridJobExecuteResponse; +import org.apache.ignite.internal.GridJobSiblingsRequest; +import org.apache.ignite.internal.GridJobSiblingsResponse; +import org.apache.ignite.internal.GridTaskCancelRequest; +import org.apache.ignite.internal.GridTaskSessionRequest; +import org.apache.ignite.internal.IgniteDiagnosticRequest; +import org.apache.ignite.internal.IgniteDiagnosticResponse; +import org.apache.ignite.internal.TxEntriesInfo; +import org.apache.ignite.internal.TxInfo; +import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta; +import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; +import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; +import org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest; +import org.apache.ignite.internal.managers.communication.CompressedMessage; +import org.apache.ignite.internal.managers.communication.ErrorMessage; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.managers.communication.GridIoSecurityAwareMessage; +import org.apache.ignite.internal.managers.communication.GridIoUserMessage; +import org.apache.ignite.internal.managers.communication.IgniteIoTestMessage; +import org.apache.ignite.internal.managers.communication.SessionChannelMessage; +import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean; +import org.apache.ignite.internal.managers.deployment.GridDeploymentRequest; +import org.apache.ignite.internal.managers.deployment.GridDeploymentResponse; +import org.apache.ignite.internal.managers.discovery.SecurityAwareCustomMessageWrapper; +import org.apache.ignite.internal.managers.encryption.ChangeCacheEncryptionRequest; +import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyRequest; +import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyResponse; +import org.apache.ignite.internal.managers.encryption.MasterKeyChangeRequest; +import org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessage; +import org.apache.ignite.internal.processors.authentication.User; +import org.apache.ignite.internal.processors.authentication.UserAcceptedMessage; +import org.apache.ignite.internal.processors.authentication.UserAuthenticateRequestMessage; +import org.apache.ignite.internal.processors.authentication.UserAuthenticateResponseMessage; +import org.apache.ignite.internal.processors.authentication.UserManagementOperation; +import org.apache.ignite.internal.processors.authentication.UserManagementOperationFinishedMessage; +import org.apache.ignite.internal.processors.authentication.UserProposedMessage; +import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage; +import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter; +import org.apache.ignite.internal.processors.cache.CacheEvictionEntry; +import org.apache.ignite.internal.processors.cache.CacheInvokeDirectResult; +import org.apache.ignite.internal.processors.cache.CacheStatisticsClearMessage; +import org.apache.ignite.internal.processors.cache.CacheStatisticsModeChangeMessage; +import org.apache.ignite.internal.processors.cache.ClientCacheChangeDiscoveryMessage; +import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage; +import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; +import org.apache.ignite.internal.processors.cache.ExchangeFailureMessage; +import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; +import org.apache.ignite.internal.processors.cache.GridCacheReturn; +import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponse; +import org.apache.ignite.internal.processors.cache.TxTimeoutOnPartitionMapExchangeChangeMessage; +import org.apache.ignite.internal.processors.cache.WalStateAckMessage; +import org.apache.ignite.internal.processors.cache.WalStateFinishMessage; +import org.apache.ignite.internal.processors.cache.WalStateProposeMessage; +import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataVersionInfo; +import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveAcceptedMessage; +import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveProposedMessage; +import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage; +import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage; +import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateAcceptedMessage; +import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest; +import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockRequest; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.GridNearUnlockRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.TransactionAttributesAwareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicApplicationAttributesAwareRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicCheckUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.NearCacheUpdates; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.UpdateErrors; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CacheGroupAffinityMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionFullCountersMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPair; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage; +import org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValue; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandlerResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotAwareMessage; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerifyResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckHandlersResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckPartitionHashesResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckProcessRequest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesFailureMessage; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesRequestMessage; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResult; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationEndRequest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationRequest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyHandlerResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreOperationResponse; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreStartRequest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotStartDiscoveryMessage; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; +import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; +import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryBatchAck; +import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; +import org.apache.ignite.internal.processors.cache.transactions.TxEntryValueHolder; +import org.apache.ignite.internal.processors.cache.transactions.TxLock; +import org.apache.ignite.internal.processors.cache.transactions.TxLocksRequest; +import org.apache.ignite.internal.processors.cache.transactions.TxLocksResponse; +import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; +import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; +import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; +import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; +import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage; +import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage; +import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; +import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; +import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage; +import org.apache.ignite.internal.processors.continuous.GridContinuousMessage; +import org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage; +import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry; +import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest; +import org.apache.ignite.internal.processors.datastreamer.DataStreamerResponse; +import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage; +import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage; +import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem; +import org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessage; +import org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessage; +import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasAckMessage; +import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasMessage; +import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateAckMessage; +import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateMessage; +import org.apache.ignite.internal.processors.query.QueryField; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest; +import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; +import org.apache.ignite.internal.processors.query.messages.GridQueryKillRequest; +import org.apache.ignite.internal.processors.query.messages.GridQueryKillResponse; +import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage; +import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage; +import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAddQueryEntityOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation; +import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnData; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsDecimalMessage; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectData; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequest; +import org.apache.ignite.internal.processors.query.stat.messages.StatisticsResponse; +import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest; +import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultResponse; +import org.apache.ignite.internal.processors.service.ServiceChangeBatchRequest; +import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResult; +import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResultBatch; +import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessId; +import org.apache.ignite.internal.processors.service.ServiceDeploymentRequest; +import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResult; +import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResultBatch; +import org.apache.ignite.internal.processors.service.ServiceUndeploymentRequest; +import org.apache.ignite.internal.util.GridByteArrayList; +import org.apache.ignite.internal.util.GridPartitionStateMap; +import org.apache.ignite.internal.util.distributed.FullMessage; +import org.apache.ignite.internal.util.distributed.InitMessage; +import org.apache.ignite.internal.util.distributed.SingleNodeMessage; +import org.apache.ignite.lang.IgniteProductVersion; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; +import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; +import org.apache.ignite.plugin.extensions.communication.MessageSerializer; +import org.apache.ignite.spi.collision.jobstealing.JobStealingRequest; +import org.apache.ignite.spi.communication.tcp.internal.TcpConnectionRequestDiscoveryMessage; +import org.apache.ignite.spi.communication.tcp.internal.TcpInverseConnectionResponseMessage; +import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; +import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; +import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage; +import org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage; +import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; +import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; +import org.apache.ignite.spi.discovery.tcp.messages.InetAddressMessage; +import org.apache.ignite.spi.discovery.tcp.messages.InetSocketAddressMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientAckResponse; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientMetricsUpdateMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientNodesMetricsMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingRequest; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingResponse; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientReconnectMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCollectionMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryConnectionCheckMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDiscardMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeRequest; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeResponse; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryLoopbackProblemMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryMetricsUpdateMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryServerOnlyCustomEventMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage; +import org.jetbrains.annotations.Nullable; + +/** */ +public class CoreMessagesProvider implements MessageFactoryProvider { + /** Node ID message type. */ + public static final short NODE_ID_MSG_TYPE = 11500; + + /** Handshake message type. */ + public static final short HANDSHAKE_MSG_TYPE = NODE_ID_MSG_TYPE + 1; + + /** Handshake wait message type. */ + public static final short HANDSHAKE_WAIT_MSG_TYPE = HANDSHAKE_MSG_TYPE + 1; + + /** Custom data marshaller. */ + private final Marshaller marsh; + + /** Class loader for the custom data marshalling. */ + private final ClassLoader clsLdr; + + /** */ + private short msgIdx; + + /** */ + private @Nullable MessageFactory factory; + + /** + * @param marsh Custom data marshaller. + * @param clsLdr Class loader for the custom data marshalling. + */ + public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { + this.marsh = marsh; + this.clsLdr = clsLdr; + } + + /** {@inheritDoc} */ + @Override public void registerAll(MessageFactory factory) { + assert this.factory == null; + + this.factory = factory; + + // [-44, 0..2, 42, 200..204, 210, 302] - Use in tests. + // [300..307, 350..352] - CalciteMessageFactory. + // [-4..-22, -30..-35, -54..-57] - SQL + // [-42..-37] - former hadoop. + // [64..71] - former IGFS. + + // [5000 - 5500]: Utility messages. Most of them originally come from Discovery. + // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. + msgIdx = 5000; + factory.register(msgIdx++, CompressedMessage::new); + register(factory, ErrorMessage.class); + register(factory, InetSocketAddressMessage.class); + register(factory, InetAddressMessage.class); + register(factory, TcpDiscoveryNode.class); + register(factory, IgniteProductVersion.class); + register(factory, DiscoveryDataPacket.class); + register(factory, GridByteArrayList.class); + register(factory, CacheVersionedValue.class); + register(factory, GridCacheVersion.class); + register(factory, GridCacheVersionEx.class); + + msgIdx = 5500; + register(factory, TcpDiscoveryCollectionMessage.class); + + // [5700 - 5800]: Schema operation messages. Most of them originally come from Discovery. + msgIdx = 5700; + register(factory, SchemaAlterTableAddColumnOperation.class); + register(factory, SchemaIndexCreateOperation.class); + register(factory, SchemaIndexDropOperation.class); + register(factory, SchemaAlterTableDropColumnOperation.class); + register(factory, SchemaAddQueryEntityOperation.class); + register(factory, QueryField.class); + register(factory, SchemaOperationStatusMessage.class); + register(factory, SchemaProposeDiscoveryMessage.class); + register(factory, SchemaFinishDiscoveryMessage.class); + + // [5900 - 6100]: TcpDiscoveryAbstractMessage. + msgIdx = 5900; + register(factory, TcpDiscoveryCheckFailedMessage.class); + register(factory, TcpDiscoveryPingRequest.class); + register(factory, TcpDiscoveryPingResponse.class); + register(factory, TcpDiscoveryClientPingRequest.class); + register(factory, TcpDiscoveryClientPingResponse.class); + register(factory, TcpDiscoveryLoopbackProblemMessage.class); + register(factory, TcpDiscoveryConnectionCheckMessage.class); + register(factory, TcpDiscoveryRingLatencyCheckMessage.class); + register(factory, TcpDiscoveryHandshakeRequest.class); + register(factory, TcpDiscoveryDiscardMessage.class); + register(factory, TcpDiscoveryHandshakeResponse.class); + register(factory, TcpDiscoveryAuthFailedMessage.class); + register(factory, TcpDiscoveryDuplicateIdMessage.class); + register(factory, TcpDiscoveryClientMetricsUpdateMessage.class); + register(factory, TcpDiscoveryMetricsUpdateMessage.class); + register(factory, TcpDiscoveryClientAckResponse.class); + register(factory, TcpDiscoveryNodeLeftMessage.class); + register(factory, TcpDiscoveryNodeFailedMessage.class); + register(factory, TcpDiscoveryStatusCheckMessage.class); + register(factory, TcpDiscoveryNodeAddFinishedMessage.class); + register(factory, TcpDiscoveryJoinRequestMessage.class); + register(factory, TcpDiscoveryCustomEventMessage.class); + register(factory, TcpDiscoveryServerOnlyCustomEventMessage.class); + register(factory, TcpDiscoveryNodeAddedMessage.class); + register(factory, TcpDiscoveryClientReconnectMessage.class); + + // [6200 - 6300]: Snapshot operation messages. Most of them originally come from Discovery. + msgIdx = 6200; + register(factory, SnapshotStartDiscoveryMessage.class); + register(factory, SnapshotCheckProcessRequest.class); + register(factory, SnapshotOperationRequest.class); + register(factory, SnapshotOperationEndRequest.class); + register(factory, SnapshotRestoreStartRequest.class); + register(factory, SnapshotOperationResponse.class); + register(factory, SnapshotHandlerResult.class); + register(factory, SnapshotCheckResponse.class); + register(factory, SnapshotPartitionsVerifyHandlerResponse.class); + register(factory, SnapshotRestoreOperationResponse.class); + register(factory, SnapshotMetadataResponse.class); + register(factory, SnapshotCheckPartitionHashesResponse.class); + register(factory, SnapshotCheckHandlersResponse.class); + register(factory, SnapshotFilesRequestMessage.class); + register(factory, SnapshotFilesFailureMessage.class); + register(factory, IncrementalSnapshotVerifyResult.class); + register(factory, IncrementalSnapshotAwareMessage.class); + + // [6400 - 6500]: Services messages. Most of them originally come from Discovery. + msgIdx = 6400; + register(factory, ServiceDeploymentProcessId.class); + register(factory, ServiceSingleNodeDeploymentResult.class); + register(factory, ServiceClusterDeploymentResult.class); + register(factory, ServiceDeploymentRequest.class); + register(factory, ServiceUndeploymentRequest.class); + register(factory, ServiceClusterDeploymentResultBatch.class); + register(factory, ServiceChangeBatchRequest.class); + register(factory, ServiceSingleNodeDeploymentResultBatch.class); + + // [6600 - 6700]: Originally Discovery's messages. + msgIdx = 6600; + + + // [6800 - 7000]: DiscoveryCustomMessage + msgIdx = 6800; + register(factory, TcpConnectionRequestDiscoveryMessage.class); + register(factory, DistributedMetaStorageUpdateMessage.class); + register(factory, DistributedMetaStorageUpdateAckMessage.class); + register(factory, DistributedMetaStorageCasMessage.class); + register(factory, DistributedMetaStorageCasAckMessage.class); + register(factory, FullMessage.class); + register(factory, InitMessage.class); + register(factory, CacheStatisticsModeChangeMessage.class); + register(factory, SecurityAwareCustomMessageWrapper.class); + register(factory, MetadataRemoveAcceptedMessage.class); + register(factory, MetadataRemoveProposedMessage.class); + register(factory, WalStateFinishMessage.class); + register(factory, WalStateProposeMessage.class); + register(factory, MetadataUpdateAcceptedMessage.class); + register(factory, TxTimeoutOnPartitionMapExchangeChangeMessage.class); + register(factory, UserAcceptedMessage.class); + register(factory, UserProposedMessage.class); + register(factory, ChangeGlobalStateFinishMessage.class); + register(factory, StopRoutineAckDiscoveryMessage.class); + register(factory, StopRoutineDiscoveryMessage.class); + register(factory, CacheAffinityChangeMessage.class); + register(factory, ClientCacheChangeDiscoveryMessage.class); + register(factory, MappingAcceptedMessage.class); + register(factory, MappingProposedMessage.class); + register(factory, ExchangeFailureMessage.class); + register(factory, CacheStatisticsClearMessage.class); + register(factory, ChangeGlobalStateMessage.class); + register(factory, ClientCacheChangeDummyDiscoveryMessage.class); + register(factory, DynamicCacheChangeBatch.class); + + // [10000 - 10200]: Transaction and lock related messages. Most of the originally comes from Communication. + msgIdx = 10000; + register(factory, TxInfo.class); + register(factory, TxEntriesInfo.class); + register(factory, TxLock.class); + register(factory, TxLocksRequest.class); + register(factory, TxLocksResponse.class); + register(factory, IgniteTxKey.class); + register(factory, IgniteTxEntry.class); + register(factory, TxEntryValueHolder.class); + register(factory, GridCacheTxRecoveryRequest.class); + register(factory, GridCacheTxRecoveryResponse.class); + register(factory, GridDistributedTxFinishRequest.class); + register(factory, GridDistributedTxFinishResponse.class); + register(factory, GridDistributedTxPrepareRequest.class); + register(factory, GridDistributedTxPrepareResponse.class); + register(factory, GridDhtTxFinishRequest.class); + register(factory, GridDhtTxFinishResponse.class); + register(factory, GridDhtTxPrepareRequest.class); + register(factory, GridDhtTxPrepareResponse.class); + register(factory, GridNearTxFinishRequest.class); + register(factory, GridNearTxFinishResponse.class); + register(factory, GridNearTxPrepareRequest.class); + register(factory, GridNearTxPrepareResponse.class); + register(factory, GridDhtLockRequest.class); + register(factory, GridDhtLockResponse.class); + register(factory, GridDhtUnlockRequest.class); + register(factory, GridNearLockRequest.class); + register(factory, GridNearLockResponse.class); + register(factory, GridNearUnlockRequest.class); + register(factory, GridDistributedLockRequest.class); + register(factory, GridDistributedLockResponse.class); + register(factory, GridDhtTxOnePhaseCommitAckRequest.class); + register(factory, TransactionAttributesAwareRequest.class); + + // [10300 - 10500]: Cache, DHT messages. + msgIdx = 10300; + register(factory, GridDhtForceKeysRequest.class); + register(factory, GridDhtForceKeysResponse.class); + register(factory, GridDhtAtomicDeferredUpdateResponse.class); + register(factory, GridDhtAtomicUpdateRequest.class); + register(factory, GridDhtAtomicUpdateResponse.class); + register(factory, GridNearAtomicFullUpdateRequest.class); + register(factory, GridDhtAtomicSingleUpdateRequest.class); + register(factory, GridNearAtomicUpdateResponse.class); + register(factory, GridNearAtomicSingleUpdateRequest.class); + register(factory, GridNearAtomicSingleUpdateInvokeRequest.class); + register(factory, GridNearAtomicSingleUpdateFilterRequest.class); + register(factory, GridNearAtomicCheckUpdateRequest.class); + register(factory, NearCacheUpdates.class); + register(factory, GridNearGetRequest.class); + register(factory, GridNearGetResponse.class); + register(factory, GridNearSingleGetRequest.class); + register(factory, GridNearSingleGetResponse.class); + register(factory, GridDhtAtomicNearResponse.class); + register(factory, GridCacheTtlUpdateRequest.class); + register(factory, GridCacheReturn.class); + register(factory, GridCacheEntryInfo.class); + register(factory, CacheInvokeDirectResult.class); + register(factory, GridCacheRawVersionedEntry.class); + register(factory, CacheEvictionEntry.class); + register(factory, CacheEntryPredicateAdapter.class); + register(factory, GridContinuousMessage.class); + register(factory, ContinuousRoutineStartResultMessage.class); + register(factory, UpdateErrors.class); + register(factory, LatchAckMessage.class); + register(factory, AtomicApplicationAttributesAwareRequest.class); + + // [10600-10800]: Affinity & partition maps. + msgIdx = 10600; + register(factory, GridDhtAffinityAssignmentRequest.class); + register(factory, GridDhtAffinityAssignmentResponse.class); + register(factory, CacheGroupAffinityMessage.class); + register(factory, ExchangeInfo.class); + register(factory, PartitionUpdateCountersMessage.class); + register(factory, CachePartitionPartialCountersMap.class); + register(factory, IgniteDhtDemandedPartitionsMap.class); + register(factory, CachePartitionFullCountersMap.class); + register(factory, GroupPartitionIdPair.class); + register(factory, IgniteDhtPartitionHistorySuppliersMap.class); + register(factory, GridPartitionStateMap.class); + register(factory, GridDhtPartitionMap.class); + register(factory, GridDhtPartitionFullMap.class); + register(factory, GridDhtPartitionExchangeId.class); + register(factory, GridCheckpointRequest.class); + register(factory, GridDhtPartitionDemandMessage.class); + register(factory, GridDhtPartitionSupplyMessage.class); + register(factory, GridDhtPartitionsFullMessage.class); + register(factory, GridDhtPartitionsSingleMessage.class); + register(factory, GridDhtPartitionsSingleRequest.class); + + // [10900-11100]: Query and SQL related messages. + msgIdx = 10900; + register(factory, GridCacheSqlQuery.class); + register(factory, GridCacheQueryRequest.class); + register(factory, GridCacheQueryResponse.class); + register(factory, GridQueryCancelRequest.class); + register(factory, GridQueryFailResponse.class); + register(factory, GridQueryNextPageRequest.class); + register(factory, GridQueryNextPageResponse.class); + register(factory, GridQueryKillRequest.class); + register(factory, GridQueryKillResponse.class); + register(factory, IndexKeyDefinition.class); + register(factory, IndexKeyTypeSettings.class); + register(factory, IndexQueryResultMeta.class); + register(factory, StatisticsKeyMessage.class); + register(factory, StatisticsDecimalMessage.class); + register(factory, StatisticsObjectData.class); + register(factory, StatisticsColumnData.class); + register(factory, StatisticsRequest.class); + register(factory, StatisticsResponse.class); + register(factory, CacheContinuousQueryBatchAck.class); + register(factory, CacheContinuousQueryEntry.class); + + // [11200 - 11300]: Compute and distributed process messages. + msgIdx = 11200; + register(factory, GridJobCancelRequest.class); + register(factory, GridJobExecuteRequest.class); + register(factory, GridJobExecuteResponse.class); + register(factory, GridJobSiblingsRequest.class); + register(factory, GridJobSiblingsResponse.class); + register(factory, GridTaskCancelRequest.class); + register(factory, GridTaskSessionRequest.class); + register(factory, GridTaskResultRequest.class); + register(factory, GridTaskResultResponse.class); + register(factory, JobStealingRequest.class); + register(factory, SingleNodeMessage.class); + + // [11500 - 11600]: IO, networking messages. + msgIdx = NODE_ID_MSG_TYPE; + register(factory, NodeIdMessage.class); + register(factory, HandshakeMessage.class); + register(factory, HandshakeWaitMessage.class); + register(factory, GridIoMessage.class); + register(factory, GridIoUserMessage.class); + register(factory, GridIoSecurityAwareMessage.class); + register(factory, RecoveryLastReceivedMessage.class); + register(factory, TcpInverseConnectionResponseMessage.class); + register(factory, SessionChannelMessage.class); + factory.register(msgIdx++, IgniteIoTestMessage::new); + + // [11700 - 11800]: Datastreamer messages. + msgIdx = 11700; + register(factory, DataStreamerUpdatesHandlerResult.class); + register(factory, DataStreamerEntry.class); + register(factory, DataStreamerRequest.class); + register(factory, DataStreamerResponse.class); + + // [11900 - 12000]: Metrics, monitoring messages. + msgIdx = 11900; + register(factory, CacheMetricsMessage.class); + register(factory, NodeMetricsMessage.class); + register(factory, NodeFullMetricsMessage.class); + register(factory, ClusterMetricsUpdateMessage.class); + register(factory, TcpDiscoveryClientNodesMetricsMessage.class); + + // [12000 - 12100]: Authentication, security messages. + msgIdx = 12000; + register(factory, User.class); + register(factory, UserManagementOperation.class); + register(factory, UserManagementOperationFinishedMessage.class); + register(factory, UserAuthenticateRequestMessage.class); + register(factory, UserAuthenticateResponseMessage.class); + + // [12200 - 12300]: Binary, classloading and marshalling messages. + msgIdx = 12200; + register(factory, GridDeploymentInfoBean.class); + register(factory, GridDeploymentRequest.class); + register(factory, GridDeploymentResponse.class); + register(factory, MissingMappingRequestMessage.class); + register(factory, MissingMappingResponseMessage.class); + register(factory, MetadataRequestMessage.class); + register(factory, MetadataResponseMessage.class); + register(factory, MarshallerMappingItem.class); + register(factory, BinaryMetadataVersionInfo.class); + + // [12400 - 12500]: Encryption messages. + msgIdx = 12400; + register(factory, GenerateEncryptionKeyRequest.class); + register(factory, GenerateEncryptionKeyResponse.class); + register(factory, ChangeCacheEncryptionRequest.class); + register(factory, MasterKeyChangeRequest.class); + + // [13000 - 13300]: Control, diagnostincs and other messages. + msgIdx = 13000; + register(factory, GridEventStorageMessage.class); + register(factory, GridChangeGlobalStateMessageResponse.class); + register(factory, IgniteDiagnosticRequest.class); + register(factory, IgniteDiagnosticResponse.class); + register(factory, WalStateAckMessage.class); + + this.factory = null; + } + + /** Registers message incrementing {@link #msgIdx}. */ + private void register(MessageFactory factory, Class cls) { + Constructor ctor; + MessageSerializer serializer; + + try { + ctor = cls.getConstructor(); + + boolean marshallable = MarshallableMessage.class.isAssignableFrom(cls); + + Class serCls = Class.forName(cls.getName() + (marshallable ? "Marshallable" : "") + "Serializer"); + + serializer = marshallable + ? (MessageSerializer)serCls.getConstructor(Marshaller.class, ClassLoader.class).newInstance(marsh, clsLdr) + : (MessageSerializer)serCls.getConstructor().newInstance(); + } + catch (Exception e) { + throw new IgniteException("Failted to register message of type " + cls.getSimpleName(), e); + } + + factory.register( + msgIdx++, + () -> { + try { + return ctor.newInstance(); + } + catch (Exception e) { + throw new IgniteException("Failted to create message of type " + cls.getSimpleName(), e); + } + }, + serializer + ); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java index 7a164d1b6d5e3..a9d0b45145820 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java @@ -80,7 +80,6 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException; -import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.IgniteDeploymentCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteKernal; @@ -129,7 +128,6 @@ import org.apache.ignite.metric.MetricRegistry; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; -import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; import org.apache.ignite.plugin.extensions.communication.MessageFormatter; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; @@ -254,9 +252,6 @@ public class GridIoManager extends GridManagerAdapter> /** Received bytes count metric name. */ public static final String RCVD_BYTES_CNT = "ReceivedBytesCount"; - /** Empty array of message factories. */ - public static final MessageFactoryProvider[] EMPTY = {}; - /** Max closed topics to store. */ public static final int MAX_CLOSED_TOPICS = 10240; @@ -442,26 +437,7 @@ public void resetMetrics() { }; } - MessageFactoryProvider[] msgs = ctx.plugins().extensions(MessageFactoryProvider.class); - - if (msgs == null) - msgs = EMPTY; - - List compMsgs = new ArrayList<>(); - - compMsgs.add(new GridIoMessageFactory(marsh, U.gridClassLoader())); - - for (IgniteComponentType compType : IgniteComponentType.values()) { - MessageFactoryProvider f = compType.messageFactory(); - - if (f != null) - compMsgs.add(f); - } - - if (!compMsgs.isEmpty()) - msgs = F.concat(msgs, compMsgs.toArray(new MessageFactoryProvider[compMsgs.size()])); - - msgFactory = new IgniteMessageFactoryImpl(msgs); + msgFactory = ctx.messageFactory(); CommunicationSpi spi = getSpi(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java deleted file mode 100644 index 6eff060a04618..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java +++ /dev/null @@ -1,581 +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.managers.communication; - -import org.apache.ignite.internal.ExchangeInfo; -import org.apache.ignite.internal.ExchangeInfoSerializer; -import org.apache.ignite.internal.GridJobCancelRequest; -import org.apache.ignite.internal.GridJobCancelRequestSerializer; -import org.apache.ignite.internal.GridJobExecuteRequest; -import org.apache.ignite.internal.GridJobExecuteRequestSerializer; -import org.apache.ignite.internal.GridJobExecuteResponse; -import org.apache.ignite.internal.GridJobExecuteResponseSerializer; -import org.apache.ignite.internal.GridJobSiblingsRequest; -import org.apache.ignite.internal.GridJobSiblingsRequestSerializer; -import org.apache.ignite.internal.GridJobSiblingsResponse; -import org.apache.ignite.internal.GridJobSiblingsResponseSerializer; -import org.apache.ignite.internal.GridTaskCancelRequest; -import org.apache.ignite.internal.GridTaskCancelRequestSerializer; -import org.apache.ignite.internal.GridTaskSessionRequest; -import org.apache.ignite.internal.GridTaskSessionRequestSerializer; -import org.apache.ignite.internal.IgniteDiagnosticRequest; -import org.apache.ignite.internal.IgniteDiagnosticRequestSerializer; -import org.apache.ignite.internal.IgniteDiagnosticResponse; -import org.apache.ignite.internal.IgniteDiagnosticResponseSerializer; -import org.apache.ignite.internal.TxEntriesInfo; -import org.apache.ignite.internal.TxEntriesInfoSerializer; -import org.apache.ignite.internal.TxInfo; -import org.apache.ignite.internal.TxInfoSerializer; -import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta; -import org.apache.ignite.internal.cache.query.index.IndexQueryResultMetaSerializer; -import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; -import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinitionSerializer; -import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; -import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettingsSerializer; -import org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest; -import org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequestSerializer; -import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean; -import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBeanSerializer; -import org.apache.ignite.internal.managers.deployment.GridDeploymentRequest; -import org.apache.ignite.internal.managers.deployment.GridDeploymentRequestSerializer; -import org.apache.ignite.internal.managers.deployment.GridDeploymentResponse; -import org.apache.ignite.internal.managers.deployment.GridDeploymentResponseSerializer; -import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyRequest; -import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyRequestSerializer; -import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyResponse; -import org.apache.ignite.internal.managers.encryption.GenerateEncryptionKeyResponseSerializer; -import org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessage; -import org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessageMarshallableSerializer; -import org.apache.ignite.internal.processors.authentication.UserAuthenticateRequestMessage; -import org.apache.ignite.internal.processors.authentication.UserAuthenticateRequestMessageSerializer; -import org.apache.ignite.internal.processors.authentication.UserAuthenticateResponseMessage; -import org.apache.ignite.internal.processors.authentication.UserAuthenticateResponseMessageSerializer; -import org.apache.ignite.internal.processors.authentication.UserManagementOperationFinishedMessage; -import org.apache.ignite.internal.processors.authentication.UserManagementOperationFinishedMessageSerializer; -import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapter; -import org.apache.ignite.internal.processors.cache.CacheEntryPredicateAdapterSerializer; -import org.apache.ignite.internal.processors.cache.CacheEvictionEntry; -import org.apache.ignite.internal.processors.cache.CacheEvictionEntrySerializer; -import org.apache.ignite.internal.processors.cache.CacheInvokeDirectResult; -import org.apache.ignite.internal.processors.cache.CacheInvokeDirectResultSerializer; -import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo; -import org.apache.ignite.internal.processors.cache.GridCacheEntryInfoSerializer; -import org.apache.ignite.internal.processors.cache.GridCacheReturn; -import org.apache.ignite.internal.processors.cache.GridCacheReturnSerializer; -import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponse; -import org.apache.ignite.internal.processors.cache.GridChangeGlobalStateMessageResponseSerializer; -import org.apache.ignite.internal.processors.cache.WalStateAckMessage; -import org.apache.ignite.internal.processors.cache.WalStateAckMessageSerializer; -import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataVersionInfo; -import org.apache.ignite.internal.processors.cache.binary.BinaryMetadataVersionInfoSerializer; -import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage; -import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessageSerializer; -import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage; -import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse; -import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishResponse; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxFinishResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareResponse; -import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxPrepareResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.GridNearUnlockRequest; -import org.apache.ignite.internal.processors.cache.distributed.GridNearUnlockRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtAffinityAssignmentResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLockResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtUnlockRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessageMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.TransactionAttributesAwareRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.TransactionAttributesAwareRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicApplicationAttributesAwareRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicApplicationAttributesAwareRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicNearResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicCheckUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicCheckUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.NearCacheUpdates; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.NearCacheUpdatesSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.UpdateErrors; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.UpdateErrorsSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CacheGroupAffinityMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CacheGroupAffinityMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionFullCountersMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionFullCountersMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeIdSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessageMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequest; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPair; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPairSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMap; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtPartitionHistorySuppliersMapSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValue; -import org.apache.ignite.internal.processors.cache.distributed.near.CacheVersionedValueSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearSingleGetResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishResponseSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequestSerializer; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse; -import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandlerResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandlerResultSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotAwareMessage; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotAwareMessageSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerifyResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerifyResultMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckHandlersResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckHandlersResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckPartitionHashesResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckPartitionHashesResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesFailureMessage; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesFailureMessageSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesRequestMessage; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFilesRequestMessageSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResultSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyHandlerResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyHandlerResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreOperationResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreOperationResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryRequestSerializer; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponse; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryResponseSerializer; -import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery; -import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuerySerializer; -import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryBatchAck; -import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryBatchAckSerializer; -import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntry; -import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryEntryMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntrySerializer; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; -import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKeySerializer; -import org.apache.ignite.internal.processors.cache.transactions.TxEntryValueHolder; -import org.apache.ignite.internal.processors.cache.transactions.TxEntryValueHolderSerializer; -import org.apache.ignite.internal.processors.cache.transactions.TxLock; -import org.apache.ignite.internal.processors.cache.transactions.TxLockSerializer; -import org.apache.ignite.internal.processors.cache.transactions.TxLocksRequest; -import org.apache.ignite.internal.processors.cache.transactions.TxLocksRequestSerializer; -import org.apache.ignite.internal.processors.cache.transactions.TxLocksResponse; -import org.apache.ignite.internal.processors.cache.transactions.TxLocksResponseSerializer; -import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntry; -import org.apache.ignite.internal.processors.cache.version.GridCacheRawVersionedEntrySerializer; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersionExSerializer; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersionSerializer; -import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; -import org.apache.ignite.internal.processors.cluster.CacheMetricsMessageSerializer; -import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessage; -import org.apache.ignite.internal.processors.cluster.ClusterMetricsUpdateMessageSerializer; -import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; -import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessageSerializer; -import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; -import org.apache.ignite.internal.processors.cluster.NodeMetricsMessageSerializer; -import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage; -import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessageSerializer; -import org.apache.ignite.internal.processors.continuous.GridContinuousMessage; -import org.apache.ignite.internal.processors.continuous.GridContinuousMessageSerializer; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntrySerializer; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequest; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerRequestSerializer; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerResponse; -import org.apache.ignite.internal.processors.datastreamer.DataStreamerResponseSerializer; -import org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessage; -import org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessageSerializer; -import org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessage; -import org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessageSerializer; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequestSerializer; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponseSerializer; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequestSerializer; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse; -import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponseSerializer; -import org.apache.ignite.internal.processors.query.messages.GridQueryKillRequest; -import org.apache.ignite.internal.processors.query.messages.GridQueryKillRequestSerializer; -import org.apache.ignite.internal.processors.query.messages.GridQueryKillResponse; -import org.apache.ignite.internal.processors.query.messages.GridQueryKillResponseSerializer; -import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessage; -import org.apache.ignite.internal.processors.query.schema.message.SchemaOperationStatusMessageSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnData; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsColumnDataSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsDecimalMessage; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsDecimalMessageSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessage; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsKeyMessageSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectData; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsObjectDataSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequest; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsRequestSerializer; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsResponse; -import org.apache.ignite.internal.processors.query.stat.messages.StatisticsResponseSerializer; -import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest; -import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequestSerializer; -import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultResponse; -import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultResponseSerializer; -import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessId; -import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessIdSerializer; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResult; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResultBatch; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResultBatchSerializer; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResultSerializer; -import org.apache.ignite.internal.util.GridByteArrayList; -import org.apache.ignite.internal.util.GridByteArrayListSerializer; -import org.apache.ignite.internal.util.GridPartitionStateMap; -import org.apache.ignite.internal.util.GridPartitionStateMapSerializer; -import org.apache.ignite.internal.util.distributed.SingleNodeMessage; -import org.apache.ignite.internal.util.distributed.SingleNodeMessageSerializer; -import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; -import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; -import org.apache.ignite.spi.collision.jobstealing.JobStealingRequest; -import org.apache.ignite.spi.collision.jobstealing.JobStealingRequestSerializer; -import org.apache.ignite.spi.communication.tcp.internal.TcpInverseConnectionResponseMessage; -import org.apache.ignite.spi.communication.tcp.internal.TcpInverseConnectionResponseMessageSerializer; -import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; -import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessageSerializer; -import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessage; -import org.apache.ignite.spi.communication.tcp.messages.HandshakeWaitMessageSerializer; -import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage; -import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessageSerializer; -import org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage; -import org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessageSerializer; - -/** - * Message factory implementation. - */ -public class GridIoMessageFactory implements MessageFactoryProvider { - /** Custom data marshaller. */ - private final Marshaller marsh; - - /** Class loader for the custom data marshalling. */ - private final ClassLoader clsLdr; - - /** - * @param marsh Custom data marshaller. - * @param clsLdr Class loader for the custom data marshalling. - */ - public GridIoMessageFactory(Marshaller marsh, ClassLoader clsLdr) { - this.marsh = marsh; - this.clsLdr = clsLdr; - } - - /** {@inheritDoc} */ - @Override public void registerAll(MessageFactory factory) { - // -54 is reserved for SQL. - // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. - factory.register(-67, CompressedMessage::new); - factory.register(-66, ErrorMessage::new, new ErrorMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(-65, TxInfo::new, new TxInfoSerializer()); - factory.register(-64, TxEntriesInfo::new, new TxEntriesInfoSerializer()); - factory.register(-63, ExchangeInfo::new, new ExchangeInfoSerializer()); - factory.register(-62, IgniteDiagnosticResponse::new, new IgniteDiagnosticResponseSerializer()); - factory.register(-61, IgniteDiagnosticRequest::new, new IgniteDiagnosticRequestSerializer()); - factory.register(-53, SchemaOperationStatusMessage::new, new SchemaOperationStatusMessageSerializer()); - factory.register(-51, NearCacheUpdates::new, new NearCacheUpdatesSerializer()); - factory.register(-50, GridNearAtomicCheckUpdateRequest::new, new GridNearAtomicCheckUpdateRequestSerializer()); - factory.register(-49, UpdateErrors::new, new UpdateErrorsSerializer()); - factory.register(-48, GridDhtAtomicNearResponse::new, new GridDhtAtomicNearResponseSerializer()); - factory.register(-45, GridChangeGlobalStateMessageResponse::new, new GridChangeGlobalStateMessageResponseSerializer()); - factory.register((short)-43, IgniteIoTestMessage::new); - factory.register(-36, GridDhtAtomicSingleUpdateRequest::new, new GridDhtAtomicSingleUpdateRequestSerializer()); - factory.register(-27, GridDhtTxOnePhaseCommitAckRequest::new, new GridDhtTxOnePhaseCommitAckRequestSerializer()); - factory.register(-25, TxLock::new, new TxLockSerializer()); - factory.register(-24, TxLocksRequest::new, new TxLocksRequestSerializer()); - factory.register(-23, TxLocksResponse::new, new TxLocksResponseSerializer()); - factory.register(-1, NodeIdMessage::new, new NodeIdMessageSerializer()); - factory.register(-2, RecoveryLastReceivedMessage::new, - new RecoveryLastReceivedMessageSerializer()); - factory.register(-3, HandshakeMessage::new, new HandshakeMessageSerializer()); - factory.register(-28, HandshakeWaitMessage::new, new HandshakeWaitMessageSerializer()); - factory.register(0, GridJobCancelRequest::new, new GridJobCancelRequestSerializer()); - factory.register(1, GridJobExecuteRequest::new, new GridJobExecuteRequestSerializer()); - factory.register(2, GridJobExecuteResponse::new, new GridJobExecuteResponseSerializer()); - factory.register(3, GridJobSiblingsRequest::new, new GridJobSiblingsRequestSerializer()); - factory.register(4, GridJobSiblingsResponse::new, new GridJobSiblingsResponseSerializer()); - factory.register(5, GridTaskCancelRequest::new, new GridTaskCancelRequestSerializer()); - factory.register(6, GridTaskSessionRequest::new, new GridTaskSessionRequestSerializer()); - factory.register(7, GridCheckpointRequest::new, new GridCheckpointRequestSerializer()); - factory.register(8, GridIoMessage::new, new GridIoMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(9, GridIoUserMessage::new, new GridIoUserMessageSerializer()); - factory.register(10, GridDeploymentInfoBean::new, new GridDeploymentInfoBeanSerializer()); - factory.register(11, GridDeploymentRequest::new, new GridDeploymentRequestSerializer()); - factory.register(12, GridDeploymentResponse::new, new GridDeploymentResponseSerializer()); - factory.register(13, GridEventStorageMessage::new, new GridEventStorageMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(16, GridCacheTxRecoveryRequest::new, new GridCacheTxRecoveryRequestSerializer()); - factory.register(17, GridCacheTxRecoveryResponse::new, new GridCacheTxRecoveryResponseSerializer()); - factory.register(18, IndexQueryResultMeta::new, new IndexQueryResultMetaSerializer()); - factory.register(19, IndexKeyTypeSettings::new, new IndexKeyTypeSettingsSerializer()); - factory.register(20, GridCacheTtlUpdateRequest::new, new GridCacheTtlUpdateRequestSerializer()); - factory.register(21, GridDistributedLockRequest::new, new GridDistributedLockRequestSerializer()); - factory.register(22, GridDistributedLockResponse::new, new GridDistributedLockResponseSerializer()); - factory.register(23, GridDistributedTxFinishRequest::new, new GridDistributedTxFinishRequestSerializer()); - factory.register(24, GridDistributedTxFinishResponse::new, new GridDistributedTxFinishResponseSerializer()); - factory.register(25, GridDistributedTxPrepareRequest::new, new GridDistributedTxPrepareRequestSerializer()); - factory.register(26, GridDistributedTxPrepareResponse::new, new GridDistributedTxPrepareResponseSerializer()); - // Type 27 is former GridDistributedUnlockRequest - factory.register(28, GridDhtAffinityAssignmentRequest::new, new GridDhtAffinityAssignmentRequestSerializer()); - factory.register(29, GridDhtAffinityAssignmentResponse::new, new GridDhtAffinityAssignmentResponseSerializer()); - factory.register(30, GridDhtLockRequest::new, new GridDhtLockRequestSerializer()); - factory.register(31, GridDhtLockResponse::new, new GridDhtLockResponseSerializer()); - factory.register(32, GridDhtTxFinishRequest::new, new GridDhtTxFinishRequestSerializer()); - factory.register(33, GridDhtTxFinishResponse::new, new GridDhtTxFinishResponseSerializer()); - factory.register(34, GridDhtTxPrepareRequest::new, new GridDhtTxPrepareRequestSerializer()); - factory.register(35, GridDhtTxPrepareResponse::new, new GridDhtTxPrepareResponseSerializer()); - factory.register(36, GridDhtUnlockRequest::new, new GridDhtUnlockRequestSerializer()); - factory.register(37, GridDhtAtomicDeferredUpdateResponse::new, new GridDhtAtomicDeferredUpdateResponseSerializer()); - factory.register(38, GridDhtAtomicUpdateRequest::new, new GridDhtAtomicUpdateRequestSerializer()); - factory.register(39, GridDhtAtomicUpdateResponse::new, new GridDhtAtomicUpdateResponseSerializer()); - factory.register(40, GridNearAtomicFullUpdateRequest::new, new GridNearAtomicFullUpdateRequestSerializer()); - factory.register(41, GridNearAtomicUpdateResponse::new, new GridNearAtomicUpdateResponseSerializer()); - factory.register(42, GridDhtForceKeysRequest::new, new GridDhtForceKeysRequestSerializer()); - factory.register(43, GridDhtForceKeysResponse::new, new GridDhtForceKeysResponseSerializer()); - factory.register(45, GridDhtPartitionDemandMessage::new, new GridDhtPartitionDemandMessageSerializer()); - factory.register(46, GridDhtPartitionsFullMessage::new, new GridDhtPartitionsFullMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(47, GridDhtPartitionsSingleMessage::new, new GridDhtPartitionsSingleMessageSerializer()); - factory.register(48, GridDhtPartitionsSingleRequest::new, new GridDhtPartitionsSingleRequestSerializer()); - factory.register(49, GridNearGetRequest::new, new GridNearGetRequestSerializer()); - factory.register(50, GridNearGetResponse::new, new GridNearGetResponseSerializer()); - factory.register(51, GridNearLockRequest::new, new GridNearLockRequestSerializer()); - factory.register(52, GridNearLockResponse::new, new GridNearLockResponseSerializer()); - factory.register(53, GridNearTxFinishRequest::new, new GridNearTxFinishRequestSerializer()); - factory.register(54, GridNearTxFinishResponse::new, new GridNearTxFinishResponseSerializer()); - factory.register(55, GridNearTxPrepareRequest::new, new GridNearTxPrepareRequestSerializer()); - factory.register(56, GridNearTxPrepareResponse::new, new GridNearTxPrepareResponseSerializer()); - factory.register(57, GridNearUnlockRequest::new, new GridNearUnlockRequestSerializer()); - factory.register(58, GridCacheQueryRequest::new, new GridCacheQueryRequestSerializer()); - factory.register(59, GridCacheQueryResponse::new, new GridCacheQueryResponseSerializer()); - factory.register(61, GridContinuousMessage::new, new GridContinuousMessageSerializer()); - factory.register(62, DataStreamerRequest::new, new DataStreamerRequestSerializer()); - factory.register(63, DataStreamerResponse::new, new DataStreamerResponseSerializer()); - factory.register(76, GridTaskResultRequest::new, new GridTaskResultRequestSerializer()); - factory.register(77, GridTaskResultResponse::new, new GridTaskResultResponseSerializer()); - factory.register(78, MissingMappingRequestMessage::new, new MissingMappingRequestMessageSerializer()); - factory.register(79, MissingMappingResponseMessage::new, new MissingMappingResponseMessageSerializer()); - factory.register(80, MetadataRequestMessage::new, new MetadataRequestMessageSerializer()); - factory.register(81, MetadataResponseMessage::new, new MetadataResponseMessageSerializer()); - factory.register(82, JobStealingRequest::new, new JobStealingRequestSerializer()); - factory.register(84, GridByteArrayList::new, new GridByteArrayListSerializer()); - factory.register(86, GridCacheVersion::new, new GridCacheVersionSerializer()); - factory.register(87, GridDhtPartitionExchangeId::new, new GridDhtPartitionExchangeIdSerializer()); - factory.register(88, GridCacheReturn::new, new GridCacheReturnSerializer()); - factory.register(91, GridCacheEntryInfo::new, new GridCacheEntryInfoSerializer()); - factory.register(93, CacheInvokeDirectResult::new, new CacheInvokeDirectResultSerializer()); - factory.register(94, IgniteTxKey::new, new IgniteTxKeySerializer()); - factory.register(95, DataStreamerEntry::new, new DataStreamerEntrySerializer()); - factory.register(96, CacheContinuousQueryEntry::new, new CacheContinuousQueryEntryMarshallableSerializer(marsh, clsLdr)); - factory.register(97, CacheEvictionEntry::new, new CacheEvictionEntrySerializer()); - factory.register(98, CacheEntryPredicateAdapter::new, new CacheEntryPredicateAdapterSerializer()); - factory.register(100, IgniteTxEntry::new, new IgniteTxEntrySerializer()); - factory.register(101, TxEntryValueHolder::new, new TxEntryValueHolderSerializer()); - factory.register(102, CacheVersionedValue::new, new CacheVersionedValueSerializer()); - factory.register(103, GridCacheRawVersionedEntry::new, new GridCacheRawVersionedEntrySerializer()); - factory.register(104, GridCacheVersionEx::new, new GridCacheVersionExSerializer()); - factory.register(106, GridQueryCancelRequest::new, new GridQueryCancelRequestSerializer()); - factory.register(107, GridQueryFailResponse::new, new GridQueryFailResponseSerializer()); - factory.register(108, GridQueryNextPageRequest::new, new GridQueryNextPageRequestSerializer()); - factory.register(109, GridQueryNextPageResponse::new, new GridQueryNextPageResponseSerializer()); - factory.register(112, GridCacheSqlQuery::new, new GridCacheSqlQuerySerializer()); - factory.register(113, IndexKeyDefinition::new, new IndexKeyDefinitionSerializer()); - factory.register(114, GridDhtPartitionSupplyMessage::new, new GridDhtPartitionSupplyMessageSerializer()); - factory.register(116, GridNearSingleGetRequest::new, new GridNearSingleGetRequestSerializer()); - factory.register(117, GridNearSingleGetResponse::new, new GridNearSingleGetResponseSerializer()); - factory.register(118, CacheContinuousQueryBatchAck::new, new CacheContinuousQueryBatchAckSerializer()); - - // [120..123] - DR - factory.register(125, GridNearAtomicSingleUpdateRequest::new, new GridNearAtomicSingleUpdateRequestSerializer()); - factory.register(126, GridNearAtomicSingleUpdateInvokeRequest::new, new GridNearAtomicSingleUpdateInvokeRequestSerializer()); - factory.register(127, GridNearAtomicSingleUpdateFilterRequest::new, new GridNearAtomicSingleUpdateFilterRequestSerializer()); - factory.register(128, CacheGroupAffinityMessage::new, new CacheGroupAffinityMessageSerializer()); - factory.register(129, WalStateAckMessage::new, new WalStateAckMessageSerializer()); - factory.register(130, UserManagementOperationFinishedMessage::new, new UserManagementOperationFinishedMessageSerializer()); - factory.register(131, UserAuthenticateRequestMessage::new, new UserAuthenticateRequestMessageSerializer()); - factory.register(132, UserAuthenticateResponseMessage::new, new UserAuthenticateResponseMessageSerializer()); - factory.register(133, ClusterMetricsUpdateMessage::new, - new ClusterMetricsUpdateMessageSerializer()); - factory.register(134, ContinuousRoutineStartResultMessage::new, new ContinuousRoutineStartResultMessageSerializer()); - factory.register(135, LatchAckMessage::new, new LatchAckMessageSerializer()); - factory.register(136, CacheMetricsMessage::new, new CacheMetricsMessageSerializer()); - factory.register(137, NodeMetricsMessage::new, new NodeMetricsMessageSerializer()); - factory.register(138, NodeFullMetricsMessage::new, new NodeFullMetricsMessageSerializer()); - factory.register(157, PartitionUpdateCountersMessage::new, new PartitionUpdateCountersMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(162, GenerateEncryptionKeyRequest::new, new GenerateEncryptionKeyRequestSerializer()); - factory.register(163, GenerateEncryptionKeyResponse::new, new GenerateEncryptionKeyResponseSerializer()); - factory.register(167, ServiceDeploymentProcessId::new, new ServiceDeploymentProcessIdSerializer()); - factory.register(168, ServiceSingleNodeDeploymentResultBatch::new, new ServiceSingleNodeDeploymentResultBatchSerializer()); - factory.register(169, ServiceSingleNodeDeploymentResult::new, new ServiceSingleNodeDeploymentResultSerializer()); - factory.register(170, GridQueryKillRequest::new, new GridQueryKillRequestSerializer()); - factory.register(171, GridQueryKillResponse::new, new GridQueryKillResponseSerializer()); - factory.register(174, GridIoSecurityAwareMessage::new, - new GridIoSecurityAwareMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(175, SessionChannelMessage::new, new SessionChannelMessageSerializer()); - factory.register(176, SingleNodeMessage::new, new SingleNodeMessageSerializer()); - factory.register(177, TcpInverseConnectionResponseMessage::new, new TcpInverseConnectionResponseMessageSerializer()); - factory.register(178, SnapshotFilesRequestMessage::new, - new SnapshotFilesRequestMessageSerializer()); - factory.register(179, SnapshotFilesFailureMessage::new, - new SnapshotFilesFailureMessageSerializer()); - factory.register(180, AtomicApplicationAttributesAwareRequest::new, new AtomicApplicationAttributesAwareRequestSerializer()); - factory.register(181, TransactionAttributesAwareRequest::new, new TransactionAttributesAwareRequestSerializer()); - - // Incremental snapshot. - factory.register(182, IncrementalSnapshotAwareMessage::new, - new IncrementalSnapshotAwareMessageSerializer()); - - // Index statistics. - factory.register(183, StatisticsKeyMessage::new, new StatisticsKeyMessageSerializer()); - factory.register(184, StatisticsDecimalMessage::new, new StatisticsDecimalMessageSerializer()); - factory.register(185, StatisticsObjectData::new, new StatisticsObjectDataSerializer()); - factory.register(186, StatisticsColumnData::new, new StatisticsColumnDataSerializer()); - factory.register(187, StatisticsRequest::new, new StatisticsRequestSerializer()); - factory.register(188, StatisticsResponse::new, new StatisticsResponseSerializer()); - - factory.register(500, CachePartitionPartialCountersMap::new, - new CachePartitionPartialCountersMapSerializer()); - factory.register(501, IgniteDhtDemandedPartitionsMap::new, - new IgniteDhtDemandedPartitionsMapSerializer()); - factory.register(505, BinaryMetadataVersionInfo::new, - new BinaryMetadataVersionInfoSerializer()); - factory.register(506, CachePartitionFullCountersMap::new, - new CachePartitionFullCountersMapSerializer()); - factory.register(508, GroupPartitionIdPair::new, new GroupPartitionIdPairSerializer()); - factory.register(510, IgniteDhtPartitionHistorySuppliersMap::new, - new IgniteDhtPartitionHistorySuppliersMapSerializer()); - factory.register(517, GridPartitionStateMap::new, new GridPartitionStateMapSerializer()); - factory.register(518, GridDhtPartitionMap::new, new GridDhtPartitionMapSerializer()); - factory.register(519, GridDhtPartitionFullMap::new, new GridDhtPartitionFullMapSerializer()); - factory.register(520, SnapshotOperationResponse::new, new SnapshotOperationResponseSerializer()); - factory.register(521, SnapshotHandlerResult::new, new SnapshotHandlerResultSerializer()); - factory.register(522, DataStreamerUpdatesHandlerResult::new, new DataStreamerUpdatesHandlerResultSerializer()); - factory.register(523, SnapshotCheckResponse::new, new SnapshotCheckResponseSerializer()); - factory.register(524, IncrementalSnapshotVerifyResult::new, - new IncrementalSnapshotVerifyResultMarshallableSerializer(marsh, clsLdr)); - factory.register(525, SnapshotRestoreOperationResponse::new, - new SnapshotRestoreOperationResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(526, SnapshotMetadataResponse::new, - new SnapshotMetadataResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(527, SnapshotCheckPartitionHashesResponse::new, - new SnapshotCheckPartitionHashesResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(528, SnapshotCheckHandlersResponse::new, new SnapshotCheckHandlersResponseSerializer()); - factory.register(530, SnapshotPartitionsVerifyHandlerResponse::new, - new SnapshotPartitionsVerifyHandlerResponseMarshallableSerializer(marsh, clsLdr)); - - // [-3..119] [124..129] [-23..-28] [-36..-55] [183..188] - this - // [120..123] - DR - // [-44, 0..2, 42, 200..204, 210, 302] - Use in tests. - // [300..307, 350..352] - CalciteMessageFactory. - // [400] - Incremental snapshot. - // [-4..-22, -30..-35, -54..-57] - SQL - // [2048..2053] - Snapshots - // [-42..-37] - former hadoop. - // [64..71] - former IGFS. - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java deleted file mode 100644 index 4960fb374c83a..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryMessageFactory.java +++ /dev/null @@ -1,371 +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.managers.discovery; - -import org.apache.ignite.internal.managers.communication.ErrorMessage; -import org.apache.ignite.internal.managers.communication.ErrorMessageMarshallableSerializer; -import org.apache.ignite.internal.managers.encryption.ChangeCacheEncryptionRequest; -import org.apache.ignite.internal.managers.encryption.ChangeCacheEncryptionRequestSerializer; -import org.apache.ignite.internal.managers.encryption.MasterKeyChangeRequest; -import org.apache.ignite.internal.managers.encryption.MasterKeyChangeRequestSerializer; -import org.apache.ignite.internal.processors.authentication.User; -import org.apache.ignite.internal.processors.authentication.UserAcceptedMessage; -import org.apache.ignite.internal.processors.authentication.UserAcceptedMessageSerializer; -import org.apache.ignite.internal.processors.authentication.UserManagementOperation; -import org.apache.ignite.internal.processors.authentication.UserManagementOperationSerializer; -import org.apache.ignite.internal.processors.authentication.UserProposedMessage; -import org.apache.ignite.internal.processors.authentication.UserProposedMessageSerializer; -import org.apache.ignite.internal.processors.authentication.UserSerializer; -import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessage; -import org.apache.ignite.internal.processors.cache.CacheAffinityChangeMessageSerializer; -import org.apache.ignite.internal.processors.cache.CacheStatisticsClearMessage; -import org.apache.ignite.internal.processors.cache.CacheStatisticsClearMessageSerializer; -import org.apache.ignite.internal.processors.cache.CacheStatisticsModeChangeMessage; -import org.apache.ignite.internal.processors.cache.CacheStatisticsModeChangeMessageSerializer; -import org.apache.ignite.internal.processors.cache.ClientCacheChangeDiscoveryMessage; -import org.apache.ignite.internal.processors.cache.ClientCacheChangeDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessage; -import org.apache.ignite.internal.processors.cache.ClientCacheChangeDummyDiscoveryMessageMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; -import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatchMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.ExchangeFailureMessage; -import org.apache.ignite.internal.processors.cache.ExchangeFailureMessageSerializer; -import org.apache.ignite.internal.processors.cache.TxTimeoutOnPartitionMapExchangeChangeMessage; -import org.apache.ignite.internal.processors.cache.TxTimeoutOnPartitionMapExchangeChangeMessageSerializer; -import org.apache.ignite.internal.processors.cache.WalStateFinishMessage; -import org.apache.ignite.internal.processors.cache.WalStateFinishMessageSerializer; -import org.apache.ignite.internal.processors.cache.WalStateProposeMessage; -import org.apache.ignite.internal.processors.cache.WalStateProposeMessageSerializer; -import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveAcceptedMessage; -import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveAcceptedMessageSerializer; -import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveProposedMessage; -import org.apache.ignite.internal.processors.cache.binary.MetadataRemoveProposedMessageSerializer; -import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateAcceptedMessage; -import org.apache.ignite.internal.processors.cache.binary.MetadataUpdateAcceptedMessageSerializer; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeIdSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandlerResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.DataStreamerUpdatesHandlerResultSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerifyResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerifyResultMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckHandlersResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckHandlersResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckPartitionHashesResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckPartitionHashesResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckProcessRequest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckProcessRequestSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotCheckResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResult; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotHandlerResultSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadataResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationEndRequest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationEndRequestSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationRequest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationRequestSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotOperationResponseSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyHandlerResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotPartitionsVerifyHandlerResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreOperationResponse; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreOperationResponseMarshallableSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreStartRequest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreStartRequestSerializer; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotStartDiscoveryMessage; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotStartDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; -import org.apache.ignite.internal.processors.cache.version.GridCacheVersionSerializer; -import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; -import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessageSerializer; -import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage; -import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessageMarshallableSerializer; -import org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage; -import org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage; -import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage; -import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessageSerializer; -import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage; -import org.apache.ignite.internal.processors.marshaller.MappingProposedMessageSerializer; -import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem; -import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItemSerializer; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasAckMessage; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasAckMessageSerializer; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasMessage; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageCasMessageSerializer; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateAckMessage; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateAckMessageSerializer; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateMessage; -import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageUpdateMessageSerializer; -import org.apache.ignite.internal.processors.query.QueryField; -import org.apache.ignite.internal.processors.query.QueryFieldMarshallableSerializer; -import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage; -import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage; -import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessageSerializer; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAddQueryEntityOperation; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAddQueryEntityOperationMarshallableSerializer; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperationSerializer; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperationSerializer; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperationMarshallableSerializer; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation; -import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperationSerializer; -import org.apache.ignite.internal.processors.service.ServiceChangeBatchRequest; -import org.apache.ignite.internal.processors.service.ServiceChangeBatchRequestSerializer; -import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResult; -import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResultBatch; -import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResultBatchSerializer; -import org.apache.ignite.internal.processors.service.ServiceClusterDeploymentResultSerializer; -import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessId; -import org.apache.ignite.internal.processors.service.ServiceDeploymentProcessIdSerializer; -import org.apache.ignite.internal.processors.service.ServiceDeploymentRequest; -import org.apache.ignite.internal.processors.service.ServiceDeploymentRequestMarshallableSerializer; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResult; -import org.apache.ignite.internal.processors.service.ServiceSingleNodeDeploymentResultSerializer; -import org.apache.ignite.internal.processors.service.ServiceUndeploymentRequest; -import org.apache.ignite.internal.processors.service.ServiceUndeploymentRequestSerializer; -import org.apache.ignite.internal.util.distributed.FullMessage; -import org.apache.ignite.internal.util.distributed.FullMessageSerializer; -import org.apache.ignite.internal.util.distributed.InitMessage; -import org.apache.ignite.internal.util.distributed.InitMessageSerializer; -import org.apache.ignite.lang.IgniteProductVersion; -import org.apache.ignite.lang.IgniteProductVersionSerializer; -import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; -import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; -import org.apache.ignite.spi.communication.tcp.internal.TcpConnectionRequestDiscoveryMessage; -import org.apache.ignite.spi.communication.tcp.internal.TcpConnectionRequestDiscoveryMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; -import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacketSerializer; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; -import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodeMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.InetAddressMessage; -import org.apache.ignite.spi.discovery.tcp.messages.InetAddressMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.InetSocketAddressMessage; -import org.apache.ignite.spi.discovery.tcp.messages.InetSocketAddressMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAuthFailedMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCacheMetricsMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCacheMetricsMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCheckFailedMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientAckResponse; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientAckResponseSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientMetricsUpdateMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientMetricsUpdateMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientNodesMetricsMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientNodesMetricsMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingRequest; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingRequestSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingResponse; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientPingResponseSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientReconnectMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientReconnectMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCollectionMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCollectionMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryConnectionCheckMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryConnectionCheckMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDiscardMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDiscardMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryDuplicateIdMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeRequest; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeRequestSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeResponse; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHandshakeResponseSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryLoopbackProblemMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryLoopbackProblemMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryMetricsUpdateMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryMetricsUpdateMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFullMetricsMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFullMetricsMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeMetricsMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeMetricsMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequestSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponseSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessageSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryServerOnlyCustomEventMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryServerOnlyCustomEventMessageMarshallableSerializer; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessageSerializer; - -/** Message factory for discovery messages. */ -public class DiscoveryMessageFactory implements MessageFactoryProvider { - /** Custom data marshaller. */ - private final Marshaller marsh; - - /** Class loader for the custom data marshalling. */ - private final ClassLoader clsLdr; - - /** - * @param marsh Custom data marshaller. - * @param clsLdr Class loader for the custom data marshalling. - */ - public DiscoveryMessageFactory(Marshaller marsh, ClassLoader clsLdr) { - this.marsh = marsh; - this.clsLdr = clsLdr; - } - - /** {@inheritDoc} */ - @Override public void registerAll(MessageFactory factory) { - factory.register(-200, TcpDiscoveryCollectionMessage::new, - new TcpDiscoveryCollectionMessageMarshallableSerializer(marsh, clsLdr)); - - factory.register(-117, TcpDiscoveryNode::new, new TcpDiscoveryNodeMarshallableSerializer(marsh, clsLdr)); - factory.register(-116, IgniteProductVersion::new, new IgniteProductVersionSerializer()); - factory.register(-115, SchemaAlterTableAddColumnOperation::new, new SchemaAlterTableAddColumnOperationSerializer()); - factory.register(-114, SchemaIndexCreateOperation::new, new SchemaIndexCreateOperationMarshallableSerializer(marsh, clsLdr)); - factory.register(-113, SchemaIndexDropOperation::new, new SchemaIndexDropOperationSerializer()); - factory.register(-112, SchemaAlterTableDropColumnOperation::new, new SchemaAlterTableDropColumnOperationSerializer()); - factory.register(-111, SchemaAddQueryEntityOperation::new, new SchemaAddQueryEntityOperationMarshallableSerializer(marsh, clsLdr)); - factory.register(-110, QueryField::new, new QueryFieldMarshallableSerializer(marsh, clsLdr)); - factory.register(-109, User::new, new UserSerializer()); - factory.register(-108, UserManagementOperation::new, new UserManagementOperationSerializer()); - factory.register(-106, DiscoveryDataPacket::new, new DiscoveryDataPacketSerializer()); - factory.register(-105, TcpDiscoveryNodeFullMetricsMessage::new, new TcpDiscoveryNodeFullMetricsMessageSerializer()); - factory.register(-104, TcpDiscoveryClientNodesMetricsMessage::new, new TcpDiscoveryClientNodesMetricsMessageSerializer()); - factory.register(-103, TcpDiscoveryCacheMetricsMessage::new, new TcpDiscoveryCacheMetricsMessageSerializer()); - factory.register(-102, TcpDiscoveryNodeMetricsMessage::new, new TcpDiscoveryNodeMetricsMessageSerializer()); - factory.register(-101, InetSocketAddressMessage::new, new InetSocketAddressMessageSerializer()); - factory.register(-100, InetAddressMessage::new, new InetAddressMessageSerializer()); - factory.register(-66, ErrorMessage::new, new ErrorMessageMarshallableSerializer(marsh, clsLdr)); - - // TcpDiscoveryAbstractMessage - factory.register(0, TcpDiscoveryCheckFailedMessage::new, new TcpDiscoveryCheckFailedMessageSerializer()); - factory.register(1, TcpDiscoveryPingRequest::new, new TcpDiscoveryPingRequestSerializer()); - factory.register(2, TcpDiscoveryPingResponse::new, new TcpDiscoveryPingResponseSerializer()); - factory.register(3, TcpDiscoveryClientPingRequest::new, new TcpDiscoveryClientPingRequestSerializer()); - factory.register(4, TcpDiscoveryClientPingResponse::new, new TcpDiscoveryClientPingResponseSerializer()); - factory.register(5, TcpDiscoveryLoopbackProblemMessage::new, new TcpDiscoveryLoopbackProblemMessageSerializer()); - factory.register(6, TcpDiscoveryConnectionCheckMessage::new, new TcpDiscoveryConnectionCheckMessageSerializer()); - factory.register(7, TcpDiscoveryRingLatencyCheckMessage::new, new TcpDiscoveryRingLatencyCheckMessageSerializer()); - factory.register(8, TcpDiscoveryHandshakeRequest::new, new TcpDiscoveryHandshakeRequestSerializer()); - factory.register(9, TcpDiscoveryDiscardMessage::new, new TcpDiscoveryDiscardMessageSerializer()); - factory.register(10, TcpDiscoveryHandshakeResponse::new, new TcpDiscoveryHandshakeResponseSerializer()); - factory.register(11, TcpDiscoveryAuthFailedMessage::new, new TcpDiscoveryAuthFailedMessageSerializer()); - factory.register(12, TcpDiscoveryDuplicateIdMessage::new, new TcpDiscoveryDuplicateIdMessageSerializer()); - factory.register(13, TcpDiscoveryClientMetricsUpdateMessage::new, new TcpDiscoveryClientMetricsUpdateMessageSerializer()); - factory.register(14, TcpDiscoveryMetricsUpdateMessage::new, new TcpDiscoveryMetricsUpdateMessageSerializer()); - factory.register(15, TcpDiscoveryClientAckResponse::new, new TcpDiscoveryClientAckResponseSerializer()); - factory.register(16, TcpDiscoveryNodeLeftMessage::new, new TcpDiscoveryNodeLeftMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(17, TcpDiscoveryNodeFailedMessage::new, new TcpDiscoveryNodeFailedMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(18, TcpDiscoveryStatusCheckMessage::new, new TcpDiscoveryStatusCheckMessageSerializer()); - factory.register(19, TcpDiscoveryNodeAddFinishedMessage::new, - new TcpDiscoveryNodeAddFinishedMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(20, TcpDiscoveryJoinRequestMessage::new, - new TcpDiscoveryJoinRequestMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(21, TcpDiscoveryCustomEventMessage::new, - new TcpDiscoveryCustomEventMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(22, TcpDiscoveryServerOnlyCustomEventMessage::new, - new TcpDiscoveryServerOnlyCustomEventMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(23, TcpConnectionRequestDiscoveryMessage::new, new TcpConnectionRequestDiscoveryMessageSerializer()); - factory.register(24, DistributedMetaStorageUpdateMessage::new, new DistributedMetaStorageUpdateMessageSerializer()); - factory.register(25, DistributedMetaStorageUpdateAckMessage::new, new DistributedMetaStorageUpdateAckMessageSerializer()); - factory.register(26, DistributedMetaStorageCasMessage::new, new DistributedMetaStorageCasMessageSerializer()); - factory.register(27, DistributedMetaStorageCasAckMessage::new, new DistributedMetaStorageCasAckMessageSerializer()); - factory.register(28, TcpDiscoveryClientReconnectMessage::new, new TcpDiscoveryClientReconnectMessageSerializer()); - factory.register(29, TcpDiscoveryNodeAddedMessage::new, new TcpDiscoveryNodeAddedMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(30, FullMessage::new, new FullMessageSerializer()); - factory.register(31, InitMessage::new, new InitMessageSerializer()); - factory.register(32, SnapshotStartDiscoveryMessage::new, new SnapshotStartDiscoveryMessageSerializer()); - factory.register(33, SnapshotCheckProcessRequest::new, new SnapshotCheckProcessRequestSerializer()); - factory.register(34, SnapshotOperationRequest::new, new SnapshotOperationRequestSerializer()); - factory.register(35, MasterKeyChangeRequest::new, new MasterKeyChangeRequestSerializer()); - factory.register(36, SnapshotOperationEndRequest::new, new SnapshotOperationEndRequestSerializer()); - factory.register(37, SnapshotRestoreStartRequest::new, new SnapshotRestoreStartRequestSerializer()); - factory.register(38, ChangeCacheEncryptionRequest::new, new ChangeCacheEncryptionRequestSerializer()); - - factory.register(86, GridCacheVersion::new, new GridCacheVersionSerializer()); - factory.register(87, GridDhtPartitionExchangeId::new, new GridDhtPartitionExchangeIdSerializer()); - - factory.register(167, ServiceDeploymentProcessId::new, new ServiceDeploymentProcessIdSerializer()); - factory.register(169, ServiceSingleNodeDeploymentResult::new, new ServiceSingleNodeDeploymentResultSerializer()); - factory.register(170, ServiceClusterDeploymentResult::new, new ServiceClusterDeploymentResultSerializer()); - - // DiscoveryCustomMessage - factory.register(500, CacheStatisticsModeChangeMessage::new, new CacheStatisticsModeChangeMessageSerializer()); - factory.register(501, SecurityAwareCustomMessageWrapper::new, - new SecurityAwareCustomMessageWrapperMarshallableSerializer(marsh, clsLdr)); - factory.register(502, MetadataRemoveAcceptedMessage::new, new MetadataRemoveAcceptedMessageSerializer()); - factory.register(503, MetadataRemoveProposedMessage::new, new MetadataRemoveProposedMessageSerializer()); - factory.register(504, SchemaProposeDiscoveryMessage::new, new SchemaProposeDiscoveryMessageSerializer()); - factory.register(505, SchemaFinishDiscoveryMessage::new, new SchemaFinishDiscoveryMessageSerializer()); - factory.register(506, WalStateFinishMessage::new, new WalStateFinishMessageSerializer()); - factory.register(507, WalStateProposeMessage::new, new WalStateProposeMessageSerializer()); - factory.register(508, MetadataUpdateAcceptedMessage::new, new MetadataUpdateAcceptedMessageSerializer()); - factory.register(509, TxTimeoutOnPartitionMapExchangeChangeMessage::new, - new TxTimeoutOnPartitionMapExchangeChangeMessageSerializer()); - factory.register(510, UserAcceptedMessage::new, new UserAcceptedMessageSerializer()); - factory.register(511, UserProposedMessage::new, new UserProposedMessageSerializer()); - factory.register(512, ChangeGlobalStateFinishMessage::new, new ChangeGlobalStateFinishMessageSerializer()); - factory.register(513, StopRoutineAckDiscoveryMessage::new, new StopRoutineAckDiscoveryMessageSerializer()); - factory.register(514, StopRoutineDiscoveryMessage::new, new StopRoutineDiscoveryMessageSerializer()); - factory.register(515, CacheAffinityChangeMessage::new, new CacheAffinityChangeMessageSerializer()); - factory.register(516, ClientCacheChangeDiscoveryMessage::new, new ClientCacheChangeDiscoveryMessageSerializer()); - factory.register(517, MappingAcceptedMessage::new, new MappingAcceptedMessageSerializer()); - factory.register(518, MappingProposedMessage::new, new MappingProposedMessageSerializer()); - factory.register(519, MarshallerMappingItem::new, new MarshallerMappingItemSerializer()); - factory.register(520, SnapshotOperationResponse::new, new SnapshotOperationResponseSerializer()); - factory.register(521, SnapshotHandlerResult::new, new SnapshotHandlerResultSerializer()); - factory.register(522, DataStreamerUpdatesHandlerResult::new, new DataStreamerUpdatesHandlerResultSerializer()); - factory.register(523, SnapshotCheckResponse::new, new SnapshotCheckResponseSerializer()); - factory.register(524, IncrementalSnapshotVerifyResult::new, - new IncrementalSnapshotVerifyResultMarshallableSerializer(marsh, clsLdr)); - factory.register(525, SnapshotRestoreOperationResponse::new, - new SnapshotRestoreOperationResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(526, SnapshotMetadataResponse::new, - new SnapshotMetadataResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(527, SnapshotCheckPartitionHashesResponse::new, - new SnapshotCheckPartitionHashesResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(528, SnapshotCheckHandlersResponse::new, new SnapshotCheckHandlersResponseSerializer()); - factory.register(530, SnapshotPartitionsVerifyHandlerResponse::new, - new SnapshotPartitionsVerifyHandlerResponseMarshallableSerializer(marsh, clsLdr)); - factory.register(531, CacheStatisticsClearMessage::new, new CacheStatisticsClearMessageSerializer()); - factory.register(532, ChangeGlobalStateMessage::new, - new ChangeGlobalStateMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(533, ClientCacheChangeDummyDiscoveryMessage::new, - new ClientCacheChangeDummyDiscoveryMessageMarshallableSerializer(marsh, clsLdr)); - factory.register(534, DynamicCacheChangeBatch::new, new DynamicCacheChangeBatchMarshallableSerializer(marsh, clsLdr)); - factory.register(535, ServiceClusterDeploymentResultBatch::new, - new ServiceClusterDeploymentResultBatchSerializer()); - factory.register(536, ServiceChangeBatchRequest::new, new ServiceChangeBatchRequestSerializer()); - factory.register(537, ServiceDeploymentRequest::new, - new ServiceDeploymentRequestMarshallableSerializer(marsh, clsLdr)); - factory.register(538, ServiceUndeploymentRequest::new, new ServiceUndeploymentRequestSerializer()); - factory.register(539, ExchangeFailureMessage::new, new ExchangeFailureMessageSerializer()); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java index b41d56110ce21..b40217bf0dcd5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java @@ -20,11 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.Nullable; @@ -50,7 +50,7 @@ public class SecurityAwareCustomMessageWrapper implements DiscoverySpiCustomMess @Order(2) byte[] msgBytes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SecurityAwareCustomMessageWrapper() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java index 00b00bf7e65df..549c813df5ae2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java @@ -20,8 +20,8 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Change cache group encryption key request. @@ -48,7 +48,7 @@ public class ChangeCacheEncryptionRequest implements Message { @Order(4) byte[] masterKeyDigest; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ChangeCacheEncryptionRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java index a6a85dfee4822..9ad784f7b3fb7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java @@ -21,9 +21,9 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Master key change request. */ public class MasterKeyChangeRequest implements Message { @@ -39,7 +39,7 @@ public class MasterKeyChangeRequest implements Message { @Order(2) byte[] digest; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public MasterKeyChangeRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java index e2747dda762ad..0e38787604a08 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java @@ -21,8 +21,8 @@ import java.util.Collections; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; @@ -58,7 +58,7 @@ public class CacheStatisticsModeChangeMessage implements DiscoveryCustomMessage, byte flags; /** - * Constructor for {@link DiscoveryMessageFactory}. + * Constructor for {@link CoreMessagesProvider}. */ public CacheStatisticsModeChangeMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java index e46e0cf6dcf5a..720f57c44890a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java @@ -25,6 +25,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; @@ -37,7 +38,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -68,7 +68,7 @@ public class ExchangeFailureMessage implements DiscoveryCustomMessage, Message { /** Actions to be done to rollback changes done before the exchange failure. */ private transient ExchangeActions exchangeRollbackActions; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ExchangeFailureMessage() { // No-op. } 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 ba26f3e15cfb3..7d47676273d35 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 @@ -19,7 +19,7 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; @@ -43,7 +43,7 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage { public int part; /** - * Empty constructor required by {@link GridIoMessageFactory}. + * Empty constructor required by {@link CoreMessagesProvider}. */ public GridDistributedTxFinishResponse() { /* No-op. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java index a12210c29cb77..fc3f766a0c6fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java @@ -22,11 +22,11 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -59,7 +59,7 @@ abstract class AbstractSnapshotOperationRequest implements Message { @Order(5) Set nodes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public AbstractSnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java index abca745ae6347..c16735839ff45 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java @@ -18,7 +18,7 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -27,7 +27,7 @@ public class DataStreamerUpdatesHandlerResult implements Message { @Order(0) boolean streamerWarning; - /** Default constructor for {@link GridIoMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public DataStreamerUpdatesHandlerResult() { // No-op. } @@ -41,5 +41,4 @@ public DataStreamerUpdatesHandlerResult(boolean streamerWarning) { public boolean streamerWarning() { return streamerWarning; } - } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java index d614f2071d303..88215dce9104c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java @@ -20,6 +20,7 @@ import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; @@ -29,7 +30,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @@ -58,7 +58,7 @@ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @Order(3) Collection exceptions; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public IncrementalSnapshotVerifyResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java index 41ab204b19355..6ae8ff9aef638 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java @@ -19,8 +19,8 @@ import java.util.Map; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckHandlersResponse implements Message { @@ -28,7 +28,7 @@ public class SnapshotCheckHandlersResponse implements Message { @Order(0) Map>> perMetaResults; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotCheckHandlersResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java index f5489867ffcfe..504a437906928 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage @Order(0) byte[] perMetaResultsBytes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotCheckPartitionHashesResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java index 791a8f559694b..5e4900a8fc65c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java @@ -20,10 +20,10 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques @Order(2) int incIdx; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotCheckProcessRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java index e3c2d12548221..fd6ea067aee39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java @@ -21,8 +21,8 @@ import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -37,7 +37,7 @@ public final class SnapshotCheckResponse implements Message { @Order(1) @Nullable Map errors; - /** Default constructor for {@link GridIoMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotCheckResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java index d734313d05eb5..d70ac1f46ac6b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -38,7 +38,7 @@ public class SnapshotHandlerResult implements Message { @Order(1) ErrorMessage errMsg; - /** Default constructor for {@link GridIoMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotHandlerResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java index d45325ad093ad..33b8f6dc3ec81 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java @@ -20,10 +20,10 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotMetadataResponse implements MarshallableMessage { @@ -34,7 +34,7 @@ public class SnapshotMetadataResponse implements MarshallableMessage { /** */ private List metadata; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotMetadataResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java index 3b24db0c1a8e8..ed3d0dacb8432 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java @@ -20,12 +20,12 @@ import java.util.List; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -48,7 +48,7 @@ public class SnapshotOperationEndRequest implements Message { @Order(2) @Nullable List warnings; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotOperationEndRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index 25aa5fb10a57d..bb93b19176809 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -21,9 +21,9 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class SnapshotOperationRequest extends AbstractSnapshotOperationRequest { @Order(7) boolean configOnly; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java index db44f5446abb5..e9b1f434c70fd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java @@ -19,7 +19,7 @@ import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -29,7 +29,7 @@ public class SnapshotOperationResponse implements Message { @Order(0) Map> hndResults; - /** Default constructor for {@link GridIoMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java index 40ea2671f3946..0435d470e0a89 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMess @Order(0) byte[] resBytes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotPartitionsVerifyHandlerResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java index 2f658cf168073..b9426cfb36f7e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java @@ -22,11 +22,11 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Snapshot operation prepare response. */ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @@ -44,7 +44,7 @@ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @Order(1) byte[] metasBytes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotRestoreOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java index 5f8b82713243c..865f954b5f1ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotRestoreStartRequest implements Message { @@ -29,7 +29,7 @@ public class SnapshotRestoreStartRequest implements Message { @Order(0) UUID reqId; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SnapshotRestoreStartRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java index c7611c0b0bee8..23fce6364f11e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.InitMessage; import org.apache.ignite.internal.util.typedef.internal.S; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT; @@ -34,7 +34,7 @@ public class SnapshotStartDiscoveryMessage extends InitMessage allNodesMetrics; - /** Default constructor. Required for {@link GridIoMessageFactory}. */ + /** Default constructor. Required for {@link CoreMessagesProvider}. */ public ClusterMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java index f34916222d334..352ec9291cefc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java @@ -21,7 +21,7 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; @@ -36,7 +36,7 @@ public class NodeFullMetricsMessage implements Message { @Order(1) public Map cachesMetricsMsgs; - /** Empty constructor for {@link GridIoMessageFactory}. */ + /** Empty constructor for {@link CoreMessagesProvider}. */ public NodeFullMetricsMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java index 3208255346ca2..e6fd484913936 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java @@ -19,7 +19,7 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; /** */ @@ -31,7 +31,7 @@ public class DistributedMetaStorageCasAckMessage extends DistributedMetaStorageU @Order(0) boolean updated; - /** Empty constructor of {@link DiscoveryMessageFactory}. */ + /** Empty constructor of {@link CoreMessagesProvider}. */ public DistributedMetaStorageCasAckMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java index 6297dc391387b..117584b9fd0b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; @@ -37,7 +37,7 @@ public class DistributedMetaStorageCasMessage extends DistributedMetaStorageUpda @Order(1) boolean matches; - /** Empty constructor for {@link DiscoveryMessageFactory}. */ + /** Empty constructor for {@link CoreMessagesProvider}. */ public DistributedMetaStorageCasMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java index 21d44ab1a671f..dd410581ac2f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; @@ -39,7 +39,7 @@ public class DistributedMetaStorageUpdateAckMessage implements DiscoveryCustomMe @Order(1) UUID reqId; - /** Empty constructor of {@link DiscoveryMessageFactory}. */ + /** Empty constructor of {@link CoreMessagesProvider}. */ public DistributedMetaStorageUpdateAckMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java index 3f4ddd5e0c494..2127197afe69a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; @@ -50,7 +50,7 @@ public class DistributedMetaStorageUpdateMessage implements DiscoveryCustomMessa @Order(3) byte[] valBytes; - /** Empty constructor for {@link DiscoveryMessageFactory}. */ + /** Empty constructor for {@link CoreMessagesProvider}. */ public DistributedMetaStorageUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java index 875432ccbfbbd..e9965f22b4d73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Collections; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -27,7 +28,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class ServiceChangeBatchRequest implements DiscoveryCustomMessage, Messag @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ServiceChangeBatchRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java index 655f70dfd45ff..5d6b775125fab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java @@ -21,11 +21,11 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -43,7 +43,7 @@ public class ServiceClusterDeploymentResult implements Message { @GridToStringInclude Map results; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ServiceClusterDeploymentResult() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java index f947d54131eec..20c4634047e91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java @@ -19,13 +19,13 @@ import java.util.Collection; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -55,7 +55,7 @@ public class ServiceClusterDeploymentResultBatch implements DiscoveryCustomMessa @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ServiceClusterDeploymentResultBatch() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java index 4d1813477ad91..a243275bd04a5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java @@ -19,12 +19,12 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -38,7 +38,7 @@ public class ServiceDeploymentRequest extends ServiceChangeAbstractRequest imple @Order(0) byte[] cfgBytes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ServiceDeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java index 5a5911c65332f..99a85aef7cb9c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java @@ -17,16 +17,16 @@ package org.apache.ignite.internal.processors.service; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** * Service undeployment request. */ public class ServiceUndeploymentRequest extends ServiceChangeAbstractRequest { - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public ServiceUndeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java index 9639f4c843ec9..6ed2194d5a572 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java @@ -21,6 +21,7 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; @@ -28,7 +29,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -63,7 +63,7 @@ public class FullMessage implements DiscoveryCustomMessage, M @Order(4) Map err; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public FullMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java index cf9412cb587d6..b7aadc67bf089 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java @@ -19,6 +19,7 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -27,7 +28,6 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class InitMessage implements Message, DiscoveryCustomM @Order(4) public boolean waitClnRes; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public InitMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java index 08c06c08fe382..38786258f74ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -50,7 +50,7 @@ public class SingleNodeMessage implements Message { @Order(3) @Nullable ErrorMessage errMsg; - /** Default constructor for {@link GridIoMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public SingleNodeMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java index b8c02b60005ad..e66fcb41f34a0 100644 --- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java @@ -27,7 +27,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.NotNull; @@ -79,7 +79,7 @@ public class IgniteProductVersion implements Comparable, E byte[] revHash; /** - * Empty constructor required by {@link Externalizable} and {@link DiscoveryMessageFactory}. + * Empty constructor required by {@link Externalizable} and {@link CoreMessagesProvider}. */ public IgniteProductVersion() { // No-op. @@ -274,11 +274,6 @@ public int compareToIgnoreTimestamp(@NotNull IgniteProductVersion o) { revHash = U.readByteArray(in); } - /** {@inheritDoc} */ - @Override public short directType() { - return -116; - } - /** {@inheritDoc} */ @Override public String toString() { String revTsStr = IgniteVersionUtils.formatBuildTimeStamp(revTs * 1000); diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java index 614418b2e369f..fd392f059f83f 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/Message.java @@ -72,9 +72,8 @@ default short directType() { var clazz = getClass(); Short type = REGISTRATIONS.get(clazz); - if (type == null) { + if (type == null) throw new IgniteException("No registration for class " + clazz.getSimpleName()); - } return type; } diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java index 32fb13d23174a..5e36086bc2d9d 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java @@ -17,6 +17,7 @@ package org.apache.ignite.plugin.extensions.communication; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.Extension; /** @@ -25,13 +26,13 @@ * Implementation of this interface is responsible for registration of all message factories in * {@link #registerAll} method. *

- * {@link #registerAll} method's call is responsibility of {@link MessageFactory} implementation. + * {@link #registerAll} method's call is responsibility of {@link CoreMessagesProvider} implementation. */ public interface MessageFactoryProvider extends Extension { /** * Registers all messages factories. See {@link MessageFactory#register}. * - * @param factory {@link MessageFactory} implementation. + * @param factory {@link CoreMessagesProvider} implementation. */ public void registerAll(MessageFactory factory); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java index 02efe024225d6..3417cbe3a415f 100755 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java @@ -264,18 +264,6 @@ public class TcpCommunicationSpi extends TcpCommunicationConfigInitializer { /** Default connections per node. */ public static final int DFLT_CONN_PER_NODE = 1; - /** Node ID message type. */ - public static final short NODE_ID_MSG_TYPE = -1; - - /** Recovery last received ID message type. */ - public static final short RECOVERY_LAST_ID_MSG_TYPE = -2; - - /** Handshake message type. */ - public static final short HANDSHAKE_MSG_TYPE = -3; - - /** Handshake wait message type. */ - public static final short HANDSHAKE_WAIT_MSG_TYPE = -28; - /** Communication metrics group name. */ public static final String COMMUNICATION_METRICS_GROUP_NAME = MetricUtils.metricName("communication", "tcp"); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java index 97a048a508ddd..6735edf6602e4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java @@ -53,6 +53,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.GridManager; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.tracing.GridTracingManager; import org.apache.ignite.internal.processors.metric.GridMetricManager; import org.apache.ignite.internal.processors.tracing.Tracing; @@ -109,7 +110,6 @@ import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.COMMUNICATION_METRICS_GROUP_NAME; import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.CONN_IDX_META; import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.CONSISTENT_ID_META; -import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.HANDSHAKE_WAIT_MSG_TYPE; import static org.apache.ignite.spi.communication.tcp.internal.CommunicationTcpUtils.handshakeTimeoutException; import static org.apache.ignite.spi.communication.tcp.internal.CommunicationTcpUtils.isRecoverableException; import static org.apache.ignite.spi.communication.tcp.internal.CommunicationTcpUtils.nodeAddresses; @@ -832,7 +832,7 @@ public GridNioServer resetNioServer() throws IgniteCheckedException { @Override public MessageSerializer serializer(short type) { // Enable sending wait message for a communication peer while context isn't initialized. - if (impl == null && type == HANDSHAKE_WAIT_MSG_TYPE) + if (impl == null && type == CoreMessagesProvider.HANDSHAKE_WAIT_MSG_TYPE) return new HandshakeWaitMessageSerializer(); return get().serializer(type); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java index efc79ce3abfd1..bf82966e14279 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; @@ -63,7 +63,7 @@ public TcpConnectionRequestDiscoveryMessage(UUID receiverNodeId, int connIdx) { this.connIdx = connIdx; } - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpConnectionRequestDiscoveryMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java index dd99c16202141..4258926d5470b 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java @@ -25,6 +25,7 @@ import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; import org.apache.ignite.internal.util.nio.ssl.GridSslMeta; import org.apache.ignite.internal.util.typedef.internal.U; @@ -32,7 +33,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; import org.apache.ignite.spi.IgniteSpiContext; -import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.spi.communication.tcp.messages.HandshakeMessage; import org.apache.ignite.spi.communication.tcp.messages.NodeIdMessage; import org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage; @@ -40,7 +40,6 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.plugin.extensions.communication.Message.DIRECT_TYPE_SIZE; -import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.HANDSHAKE_WAIT_MSG_TYPE; import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.makeMessageType; import static org.apache.ignite.spi.communication.tcp.messages.RecoveryLastReceivedMessage.NEED_WAIT; @@ -159,10 +158,10 @@ private abstract static class BlockingTransport { if (readBytes >= DIRECT_TYPE_SIZE) { short msgType = makeMessageType(buf.get(0), buf.get(1)); - if (msgType == HANDSHAKE_WAIT_MSG_TYPE) + if (msgType == CoreMessagesProvider.HANDSHAKE_WAIT_MSG_TYPE) return null; - assert msgType == TcpCommunicationSpi.NODE_ID_MSG_TYPE; + assert msgType == CoreMessagesProvider.NODE_ID_MSG_TYPE; } totalBytes += readBytes; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java index c18760dcb5640..789f3d0adb107 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java @@ -39,6 +39,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.cache.CacheMetricsSnapshot; import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; +import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.processors.tracing.NoopTracing; import org.apache.ignite.internal.processors.tracing.Tracing; @@ -53,7 +54,6 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryClientNodesMetricsMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryMetricsUpdateMessage; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFullMetricsMessage; import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRingLatencyCheckMessage; import org.jetbrains.annotations.Nullable; @@ -409,7 +409,7 @@ protected void clearNodeSensitiveData(TcpDiscoveryNode node) { /** */ public void processCacheMetricsMessage(TcpDiscoveryMetricsUpdateMessage msg, long tsNanos) { - for (Map.Entry e : msg.serversFullMetricsMessages().entrySet()) { + for (Map.Entry e : msg.serversFullMetricsMessages().entrySet()) { UUID srvrId = e.getKey(); Map cacheMetricsMsgs = e.getValue().cachesMetricsMessages(); NodeMetricsMessage srvrMetricsMsg = e.getValue().nodeMetricsMessage(); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index c8cd42dcc3dc8..724472761bdf5 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -56,8 +56,6 @@ import org.apache.ignite.failure.FailureContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.failure.FailureProcessor; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; @@ -75,7 +73,6 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; -import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.resources.LoggerResource; import org.apache.ignite.spi.IgniteSpiAdapter; @@ -598,7 +595,7 @@ public void setClientReconnectDisabled(boolean clientReconnectDisabled) { setLocalAddress(ignite.configuration().getLocalHost()); setAddressResolver(ignite.configuration().getAddressResolver()); - marsh = ((IgniteEx)ignite).context().marshallerContext().jdkMarshaller(); + marsh = ((IgniteEx)ignite).context().marshaller(); } } @@ -2134,8 +2131,7 @@ protected void onExchange(DiscoveryDataPacket dataPacket, ClassLoader clsLdr) { registerMBean(igniteInstanceName, new TcpDiscoverySpiMBeanImpl(this), TcpDiscoverySpiMBean.class); - msgFactory = new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] { new DiscoveryMessageFactory(marshaller(), U.resolveClassLoader(ignite().configuration())) }); + msgFactory = ((IgniteEx)ignite).context().messageFactory(); impl.spiStart(igniteInstanceName); } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java index 0630144dc753b..5d21dade1d772 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java @@ -38,6 +38,7 @@ import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.managers.discovery.IgniteClusterNode; +import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -50,7 +51,6 @@ import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; -import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeMetricsMessage; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID; @@ -112,7 +112,7 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Ignite /** Node metrics message. */ @GridToStringExclude @Order(6) - volatile TcpDiscoveryNodeMetricsMessage metricsMsg; + volatile NodeMetricsMessage metricsMsg; /** Node cache metrics. */ @GridToStringExclude @@ -227,7 +227,7 @@ public TcpDiscoveryNode(UUID id, if (consistentId != null) consistentIdBytes = U.marshal(marsh, consistentId); - metricsMsg = new TcpDiscoveryNodeMetricsMessage(metrics); + metricsMsg = new NodeMetricsMessage(metrics); } /** {@inheritDoc} */ @@ -246,11 +246,6 @@ public TcpDiscoveryNode(UUID id, metricsMsg = null; } - /** {@inheritDoc} */ - @Override public short directType() { - return -117; - } - /** * @return Last successfully connected address. */ diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java index f7472e1e17d91..3d905dad3a536 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java @@ -21,7 +21,7 @@ import java.net.UnknownHostException; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -35,7 +35,7 @@ public class InetAddressMessage implements Message { @Order(1) byte[] addrBytes; - /** Default constructor for {@link DiscoveryMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public InetAddressMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java index a24b50d8b7f56..e792b9b21e8f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java @@ -19,7 +19,7 @@ import java.net.InetAddress; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; /** Socket address utility container message. Is not a pure {@link TcpDiscoveryAbstractMessage}. */ @@ -29,7 +29,7 @@ public class InetSocketAddressMessage extends InetAddressMessage { int port; /** - * Default constructor for {@link DiscoveryMessageFactory}. + * Default constructor for {@link CoreMessagesProvider}. */ public InetSocketAddressMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java index 3e633f4d47b3d..01d937cde8028 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java @@ -20,7 +20,7 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.tracing.messages.SpanContainer; import org.apache.ignite.internal.processors.tracing.messages.TraceableMessage; import org.apache.ignite.marshaller.Marshaller; @@ -40,7 +40,7 @@ public abstract class TcpDiscoveryAbstractTraceableMessage extends TcpDiscoveryA @Nullable byte[] spanContainerBytes; /** - * Default constructor for {@link DiscoveryMessageFactory}. + * Default constructor for {@link CoreMessagesProvider}. */ protected TcpDiscoveryAbstractTraceableMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java index 98d6378245f7e..c7ee4aaececc4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java @@ -20,7 +20,7 @@ import java.net.InetAddress; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -39,7 +39,7 @@ public class TcpDiscoveryAuthFailedMessage extends TcpDiscoveryAbstractMessage i @Order(1) UUID targetNodeId; - /** Default constructor for {@link DiscoveryMessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryAuthFailedMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCacheMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCacheMetricsMessage.java deleted file mode 100644 index 9f5b20c35f6e4..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCacheMetricsMessage.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.spi.discovery.tcp.messages; - -import org.apache.ignite.cache.CacheMetrics; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; -import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * We cannot directly reuse {@link CacheMetricsMessage} in Discovery as it is registered in a message factory of - * Communication component and thus is unavailable in Discovery. We have to extend {@link CacheMetricsMessage} and - * register this subclass in message factory of Discovery component. - */ -public class TcpDiscoveryCacheMetricsMessage extends CacheMetricsMessage { - /** Constructor for {@link DiscoveryMessageFactory}. */ - public TcpDiscoveryCacheMetricsMessage() { - // No-op. - } - - /** @param cacheMetricsMsg Cache metric message. */ - public TcpDiscoveryCacheMetricsMessage(CacheMetrics cacheMetricsMsg) { - super(cacheMetricsMsg); - } - - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(TcpDiscoveryCacheMetricsMessage.class, this, "super", super.toString()); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java index 0de0f136b8175..515e07397fe63 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java @@ -20,7 +20,8 @@ import java.util.UUID; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -35,9 +36,9 @@ public class TcpDiscoveryClientMetricsUpdateMessage extends TcpDiscoveryAbstract /** */ @Order(0) - TcpDiscoveryNodeMetricsMessage metricsMsg; + NodeMetricsMessage metricsMsg; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryClientMetricsUpdateMessage() { // No-op. } @@ -51,7 +52,7 @@ public TcpDiscoveryClientMetricsUpdateMessage() { public TcpDiscoveryClientMetricsUpdateMessage(UUID creatorNodeId, ClusterMetrics metrics) { super(creatorNodeId); - metricsMsg = new TcpDiscoveryNodeMetricsMessage(metrics); + metricsMsg = new NodeMetricsMessage(metrics); } /** @@ -59,7 +60,7 @@ public TcpDiscoveryClientMetricsUpdateMessage(UUID creatorNodeId, ClusterMetrics * * @return Metrics holder message. */ - public TcpDiscoveryNodeMetricsMessage metricsMessage() { + public NodeMetricsMessage metricsMessage() { return metricsMsg; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java index 78fc2cc26ae5c..140d3b9977b5b 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java @@ -20,7 +20,8 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -28,20 +29,20 @@ public class TcpDiscoveryClientNodesMetricsMessage implements Message { /** Map of nodes metrics messages per node id. */ @Order(0) - Map nodesMetricsMsgs; + Map nodesMetricsMsgs; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryClientNodesMetricsMessage() { // No-op. } /** @return Map of nodes metrics messages per node id. */ - public Map nodesMetricsMessages() { + public Map nodesMetricsMessages() { return nodesMetricsMsgs; } /** @param nodesMetricsMsgs Map of nodes metrics messages per node id. */ - public void nodesMetricsMessages(Map nodesMetricsMsgs) { + public void nodesMetricsMessages(Map nodesMetricsMsgs) { this.nodesMetricsMsgs = nodesMetricsMsgs; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java index dc1967fe98794..ffda34cf60c00 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java @@ -22,7 +22,7 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; @@ -49,7 +49,7 @@ public class TcpDiscoveryClientReconnectMessage extends TcpDiscoveryAbstractMess @Order(2) @Nullable TcpDiscoveryCollectionMessage pendingMsgsMsg; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryClientReconnectMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java index 27c14b411bf84..eea757b4a34a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -52,7 +52,7 @@ public class TcpDiscoveryCollectionMessage implements MarshallableMessage { @GridToStringExclude @Nullable byte[] marshallableMsgsBytes; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryCollectionMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java index 4814c7ee4d5a1..dbea16218c2eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java @@ -17,7 +17,7 @@ package org.apache.ignite.spi.discovery.tcp.messages; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; @@ -32,7 +32,7 @@ public class TcpDiscoveryConnectionCheckMessage extends TcpDiscoveryAbstractMess private static final long serialVersionUID = 0L; /** - * Default constructor for {@link DiscoveryMessageFactory}. + * Default constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryConnectionCheckMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java index 381a80995c3ef..17046ada00aed 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java @@ -21,7 +21,7 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.IncompleteDeserializationException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -53,7 +53,7 @@ public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractTraceabl volatile @Nullable Message serMsg; /** - * Constructor for {@link DiscoveryMessageFactory}. + * Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryCustomEventMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java index e2164d17a63be..9e503b87594af 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java @@ -19,7 +19,7 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -40,7 +40,7 @@ public class TcpDiscoveryHandshakeRequest extends TcpDiscoveryAbstractMessage im @Nullable String dcId; /** - * Default constructor for {@link DiscoveryMessageFactory}. + * Default constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryHandshakeRequest() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java index 6332fd068062f..21f207e6fc6b3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java @@ -21,7 +21,7 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -47,7 +47,7 @@ public class TcpDiscoveryHandshakeResponse extends TcpDiscoveryAbstractMessage i @Nullable Collection redirectAddrsMsgs; /** - * Default constructor for {@link DiscoveryMessageFactory}. + * Default constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryHandshakeResponse() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java index 59e726dc897a1..1f4213a92191b 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java @@ -25,8 +25,10 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; +import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; +import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; @@ -60,13 +62,13 @@ public class TcpDiscoveryMetricsUpdateMessage extends TcpDiscoveryAbstractMessag /** Servers full metrics: server id -> server metrics + metrics of server's caches. */ @GridToStringExclude @Order(1) - @Nullable Map serversFullMetricsMsgs; + @Nullable Map serversFullMetricsMsgs; /** Client node IDs. */ @Order(2) @Nullable Set clientNodeIds; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryMetricsUpdateMessage() { // No-op. } @@ -97,9 +99,9 @@ public void addServerMetrics(UUID srvrId, ClusterMetrics newMetrics) { serversFullMetricsMsgs.compute(srvrId, (srvrId0, srvrFullMetrics) -> { if (srvrFullMetrics == null) - srvrFullMetrics = new TcpDiscoveryNodeFullMetricsMessage(); + srvrFullMetrics = new NodeFullMetricsMessage(); - srvrFullMetrics.nodeMetricsMessage(new TcpDiscoveryNodeMetricsMessage(newMetrics)); + srvrFullMetrics.nodeMetricsMessage(new NodeMetricsMessage(newMetrics)); return srvrFullMetrics; }); @@ -122,12 +124,12 @@ public void addServerCacheMetrics(UUID srvrId, Map newCac serversFullMetricsMsgs.compute(srvrId, (srvrId0, srvrFullMetrics) -> { if (srvrFullMetrics == null) - srvrFullMetrics = new TcpDiscoveryNodeFullMetricsMessage(); + srvrFullMetrics = new NodeFullMetricsMessage(); Map newCachesMsgsMap = U.newHashMap(newCachesMetrics.size()); newCachesMetrics.forEach((cacheId, cacheMetrics) -> - newCachesMsgsMap.put(cacheId, new TcpDiscoveryCacheMetricsMessage(cacheMetrics))); + newCachesMsgsMap.put(cacheId, new CacheMetricsMessage(cacheMetrics))); srvrFullMetrics.cachesMetricsMessages(newCachesMsgsMap); @@ -159,7 +161,7 @@ public void addClientMetrics(UUID srvrId, UUID clientNodeId, ClusterMetrics clie clientsMetricsMsg.nodesMetricsMessages(new HashMap<>()); } - clientsMetricsMsg.nodesMetricsMessages().put(clientNodeId, new TcpDiscoveryNodeMetricsMessage(clientMetrics)); + clientsMetricsMsg.nodesMetricsMessages().put(clientNodeId, new NodeMetricsMessage(clientMetrics)); return clientsMetricsMsg; }); @@ -178,7 +180,7 @@ public void removeServerMetrics(UUID srvrId) { } /** @return Map of server full metrics messages. */ - public Map serversFullMetricsMessages() { + public Map serversFullMetricsMessages() { return serversFullMetricsMsgs; } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java index ce375a3010958..4b4b148af9091 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java @@ -21,7 +21,7 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -69,7 +69,7 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractTraceableM @Order(5) long gridStartTime; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryNodeAddedMessage() { // No-op. } @@ -222,11 +222,6 @@ public long gridStartTime() { return gridStartTime; } - /** {@inheritDoc} */ - @Override public short directType() { - return 29; - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(TcpDiscoveryNodeAddedMessage.class, this, "super", super.toString()); diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFullMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFullMetricsMessage.java deleted file mode 100644 index be70dfb74bd07..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFullMetricsMessage.java +++ /dev/null @@ -1,40 +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.spi.discovery.tcp.messages; - -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; -import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * We cannot directly reuse {@link NodeFullMetricsMessage} in Discovery as it is registered in a message factory of - * Communication component and thus is unavailable in Discovery. We have to extend {@link NodeFullMetricsMessage} and - * register this subclass in message factory of Discovery component. - */ -public class TcpDiscoveryNodeFullMetricsMessage extends NodeFullMetricsMessage { - /** Constructor for {@link DiscoveryMessageFactory}. */ - public TcpDiscoveryNodeFullMetricsMessage() { - // No-op. - } - - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(TcpDiscoveryNodeFullMetricsMessage.class, this, "super", super.toString()); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java index d271909d55335..9e50d472ddfce 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java @@ -18,7 +18,7 @@ package org.apache.ignite.spi.discovery.tcp.messages; import java.util.UUID; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -31,7 +31,7 @@ public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractTraceableMe /** */ private static final long serialVersionUID = 0L; - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryNodeLeftMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeMetricsMessage.java deleted file mode 100644 index 24c82452b6d19..0000000000000 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeMetricsMessage.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.spi.discovery.tcp.messages; - -import org.apache.ignite.cluster.ClusterMetrics; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; -import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * We cannot directly reuse {@link NodeMetricsMessage} in Discovery as it is registered in a message factory of - * Communication component and thus is unavailable in Discovery. We have to extend {@link NodeMetricsMessage} and - * register this subclass in message factory of Discovery component. - */ -public class TcpDiscoveryNodeMetricsMessage extends NodeMetricsMessage { - /** Constructor for {@link DiscoveryMessageFactory}. */ - public TcpDiscoveryNodeMetricsMessage() { - // No-op. - } - - /** @param nodeMetrics Node metrics. */ - public TcpDiscoveryNodeMetricsMessage(ClusterMetrics nodeMetrics) { - super(nodeMetrics); - } - - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(TcpDiscoveryNodeMetricsMessage.class, this, "super", super.toString()); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java index 977275ffa9c1b..6067262b4d0eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java @@ -19,7 +19,7 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; @@ -38,7 +38,7 @@ public class TcpDiscoveryRingLatencyCheckMessage extends TcpDiscoveryAbstractMes @Order(1) int curHops; - /** Empty constructor for {@link DiscoveryMessageFactory}. */ + /** Empty constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryRingLatencyCheckMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java index bdc69e356e9d5..14fe94552ab4b 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java @@ -23,7 +23,7 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; @@ -58,7 +58,7 @@ public class TcpDiscoveryStatusCheckMessage extends TcpDiscoveryAbstractMessage @Order(2) int status; - /** Empty constructor for {@link DiscoveryMessageFactory}. */ + /** Empty constructor for {@link CoreMessagesProvider}. */ public TcpDiscoveryStatusCheckMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java index cd7fccd16fd43..30c44be358a39 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; @@ -46,7 +46,7 @@ public class DirectMarshallingMessagesTest extends GridCommonAbstractTest { /** Message factory. */ private final MessageFactory msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new GridIoMessageFactory(jdk(), U.gridClassLoader()), + new CoreMessagesProvider(jdk(), U.gridClassLoader()), factory -> factory.register( TestNestedContainersMessage.TYPE, TestNestedContainersMessage::new, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java index 436408639ce94..5f1a56622ede0 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java @@ -20,8 +20,8 @@ import java.util.List; import java.util.Map; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ class TestNestedContainersMessage implements Message { @@ -40,7 +40,7 @@ class TestNestedContainersMessage implements Message { @Order(2) Map nestedArr; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public TestNestedContainersMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java index 8b76d8e2f43d5..b146af2c53967 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java @@ -26,6 +26,7 @@ import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.direct.state.DirectMessageState; import org.apache.ignite.internal.direct.stream.DirectByteBufferStream; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPair; @@ -46,7 +47,7 @@ public class CompressedMessageTest { @Test public void testWriteReadHugeMessage() { MessageFactory msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{ - new GridIoMessageFactory(jdk(), U.gridClassLoader())}); + new CoreMessagesProvider(jdk(), U.gridClassLoader())}); DirectMessageWriter writer = new DirectMessageWriter(msgFactory); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java index 3f2a942ddb1ee..e5be7dd50c8d2 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java @@ -19,6 +19,7 @@ import java.util.UUID; import org.apache.commons.lang3.reflect.FieldUtils; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; @@ -33,7 +34,7 @@ public class IgniteIoCommunicationMessageSerializationTest extends AbstractMessageSerializationTest { /** {@inheritDoc} */ @Override protected MessageFactoryProvider messageFactory() { - return new GridIoMessageFactory(jdk(), U.gridClassLoader()); + return new CoreMessagesProvider(jdk(), U.gridClassLoader()); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java index 85ad6567d5e60..53a4cad334664 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; @@ -31,7 +32,7 @@ import static org.junit.Assert.assertTrue; /** - * Tests for default implementation of {@link MessageFactory} interface. + * Tests for default implementation of {@link CoreMessagesProvider} interface. */ public class IgniteMessageFactoryImplTest { /** Test message 1 type. */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java index 7746e22f4a95b..4197222ab2caf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java @@ -21,6 +21,7 @@ import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.AbstractTestPluginProvider; import org.apache.ignite.plugin.ExtensionRegistry; import org.apache.ignite.plugin.PluginContext; @@ -29,7 +30,6 @@ import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; import org.apache.ignite.plugin.extensions.communication.MessageReader; import org.apache.ignite.plugin.extensions.communication.MessageWriter; -import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; import org.junit.Test; @@ -40,8 +40,8 @@ * for which message factory is already registered. */ public class MessageDirectTypeIdConflictTest extends GridCommonAbstractTest { - /** Message direct type. Message with this direct type will be registered by {@link GridIoMessageFactory} first. */ - private static final short MSG_DIRECT_TYPE = TcpCommunicationSpi.HANDSHAKE_MSG_TYPE; + /** Message direct type. Message with this direct type will be registered by {@link CoreMessagesProvider} first. */ + private static final short MSG_DIRECT_TYPE = CoreMessagesProvider.HANDSHAKE_MSG_TYPE; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java index b37398fa1393b..4dc2ddedd0b34 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.managers.discovery; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.AbstractMessageSerializationTest; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; @@ -27,6 +28,6 @@ public class IgniteDiscoveryMessageSerializationTest extends AbstractMessageSerializationTest { /** {@inheritDoc} */ @Override protected MessageFactoryProvider messageFactory() { - return new DiscoveryMessageFactory(jdk(), U.gridClassLoader()); + return new CoreMessagesProvider(jdk(), U.gridClassLoader()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java index e75ed12f40019..2473dacf942e5 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java @@ -32,7 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; @@ -150,7 +150,7 @@ public void testCacheContinuousQueryEntrySerialization() { e0.markFiltered(); IgniteMessageFactoryImpl msgFactory = - new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new GridIoMessageFactory(jdk(), U.gridClassLoader())}); + new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), U.gridClassLoader())}); ByteBuffer buf = ByteBuffer.allocate(4096); DirectMessageWriter writer = new DirectMessageWriter(msgFactory); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/MessagesPluginProvider.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/MessagesPluginProvider.java index 7e5b53492dc4e..8eea037502c9b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/MessagesPluginProvider.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/MessagesPluginProvider.java @@ -17,8 +17,8 @@ package org.apache.ignite.internal.util.distributed; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.AbstractTestPluginProvider; import org.apache.ignite.plugin.ExtensionRegistry; @@ -56,7 +56,7 @@ public static class MessagesInjectedTcpDiscoverySpi extends TcpDiscoverySpi { /** {@inheritDoc} */ @Override protected void initLocalNode(int srvPort, boolean addExtAddrAttr) { GridTestUtils.setFieldValue(this, TcpDiscoverySpi.class, "msgFactory", new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] { new DiscoveryMessageFactory(jdk(), U.gridClassLoader()), FACTORY_PROVIDER})); + new MessageFactoryProvider[] { new CoreMessagesProvider(jdk(), U.gridClassLoader()), FACTORY_PROVIDER})); super.initLocalNode(srvPort, addExtAddrAttr); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java index 6f148b3c98ff0..90ef3a9004e4c 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.util.distributed; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class TestIntegerMessage implements Message { @@ -27,7 +27,7 @@ public class TestIntegerMessage implements Message { @Order(0) int val; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public TestIntegerMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java index f23b35beae407..4dc4ebee4bd14 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java @@ -19,8 +19,8 @@ import java.util.UUID; import org.apache.ignite.internal.Order; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; -import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class TestUuidMessage implements Message { @@ -28,7 +28,7 @@ public class TestUuidMessage implements Message { @Order(0) UUID val; - /** Default constructor for {@link MessageFactory}. */ + /** Default constructor for {@link CoreMessagesProvider}. */ public TestUuidMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index b6256b6dc7cd9..10615b39022a4 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.typedef.F; @@ -154,7 +154,7 @@ private void startSpis() throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new GridIoMessageFactory(jdk(), U.gridClassLoader()), customMessageFactory()})); + new CoreMessagesProvider(jdk(), U.gridClassLoader()), customMessageFactory()})); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 654e72892808d..31a4f880344b8 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -37,7 +37,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteNodeAttributes; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -434,7 +434,7 @@ private void startSpis(MessageListener lsnr) throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java index d4f53758320ea..4f448c0f3c1b5 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java @@ -27,7 +27,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; import org.apache.ignite.internal.util.typedef.F; @@ -252,7 +252,7 @@ private TcpCommunicationSpi initializeSpi(GridSpiTestContext ctx, node.setId(rsrcs.getNodeId()); ctx.messageFactory(new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{ - new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY})); + new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY})); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 18909f7add40c..62fcd6d55569c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -38,7 +38,7 @@ import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.nio.GridCommunicationClient; @@ -468,7 +468,7 @@ private int getSpiCount() { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.timeoutProcessor(timeoutProcessor); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index 91343407e1af7..648d7c3febb5c 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -400,7 +400,7 @@ private void startSpis(int ackCnt, int idleTimeout, int queueLimit) throws Excep GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index 46bac32c8ff2b..9e1c6145459b7 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -32,7 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -729,7 +729,7 @@ private void startSpis() throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java index 41e925a8093b0..70fa1b6c967b3 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java @@ -21,6 +21,7 @@ import java.net.Socket; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.ListeningTestLogger; import org.apache.ignite.testframework.LogListener; @@ -29,7 +30,6 @@ import org.junit.Assert; import org.junit.Test; -import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.NODE_ID_MSG_TYPE; import static org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi.makeMessageType; /** @@ -83,7 +83,7 @@ public void clientCanNotSendHandshakeWaitMessage() throws Exception { short respMsgType = makeMessageType(b[0], b[1]); // Client can't give HANDSHAKE_WAIT_MSG_TYPE. - Assert.assertEquals(NODE_ID_MSG_TYPE, respMsgType); + Assert.assertEquals(CoreMessagesProvider.NODE_ID_MSG_TYPE, respMsgType); } } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index a1eff7105d7f3..71aaae35c5f38 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -453,7 +453,7 @@ private void startSpis(int ackCnt, int idleTimeout, int queueLimit) throws Excep GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new GridIoMessageFactory(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index 8809b1c204b41..62391811edc5e 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -40,7 +40,7 @@ import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.communication.GridIoMessageFactory; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridIoUserMessage; import org.apache.ignite.internal.managers.communication.GridMessageListener; @@ -554,7 +554,7 @@ public void triggerEvent(Event evt) { /** {@inheritDoc} */ @Override public MessageFactory messageFactory() { if (factory == null) - factory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new GridIoMessageFactory(jdk(), U.gridClassLoader())}); + factory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), U.gridClassLoader())}); return factory; } diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java index ea83271c9f54d..77b75d6922694 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java @@ -28,8 +28,8 @@ import java.util.zip.InflaterInputStream; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; +import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.jdk.JdkMarshaller; @@ -65,8 +65,7 @@ public class DiscoveryMessageParser { /** */ public DiscoveryMessageParser(Marshaller marsh) { this.marsh = marsh; - this.msgFactory = new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] { new DiscoveryMessageFactory(marsh, U.gridClassLoader()) }); + msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { new CoreMessagesProvider(marsh, U.gridClassLoader()) }); } /** Marshals discovery message to bytes array. */ From 6146ef10f934409f68ee72c9162768db06165e47 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 14:17:58 +0300 Subject: [PATCH 2/9] + master, checkstyle, cleanup --- .../{managers => }/CoreMessagesProvider.java | 75 ++++++++----------- .../apache/ignite/internal/ExchangeInfo.java | 1 - .../internal/IgniteDiagnosticRequest.java | 1 - .../internal/IgniteDiagnosticResponse.java | 1 - .../apache/ignite/internal/IgniteKernal.java | 1 - .../apache/ignite/internal/TxEntriesInfo.java | 1 - .../org/apache/ignite/internal/TxInfo.java | 1 - .../SecurityAwareCustomMessageWrapper.java | 4 +- .../ChangeCacheEncryptionRequest.java | 4 +- .../encryption/MasterKeyChangeRequest.java | 4 +- .../CacheStatisticsModeChangeMessage.java | 4 +- .../cache/ExchangeFailureMessage.java | 4 +- .../GridDistributedTxFinishResponse.java | 2 +- .../AbstractSnapshotOperationRequest.java | 4 +- .../DataStreamerUpdatesHandlerResult.java | 4 +- .../IncrementalSnapshotVerifyResult.java | 4 +- .../SnapshotCheckHandlersResponse.java | 4 +- .../SnapshotCheckPartitionHashesResponse.java | 4 +- .../snapshot/SnapshotCheckProcessRequest.java | 4 +- .../snapshot/SnapshotCheckResponse.java | 4 +- .../snapshot/SnapshotHandlerResult.java | 4 +- .../snapshot/SnapshotMetadataResponse.java | 4 +- .../snapshot/SnapshotOperationEndRequest.java | 4 +- .../snapshot/SnapshotOperationRequest.java | 4 +- .../snapshot/SnapshotOperationResponse.java | 4 +- ...apshotPartitionsVerifyHandlerResponse.java | 4 +- .../SnapshotRestoreOperationResponse.java | 4 +- .../snapshot/SnapshotRestoreStartRequest.java | 4 +- .../SnapshotStartDiscoveryMessage.java | 4 +- .../cluster/CacheMetricsMessage.java | 4 +- .../cluster/ClusterMetricsUpdateMessage.java | 4 +- .../cluster/NodeFullMetricsMessage.java | 4 +- .../DistributedMetaStorageCasAckMessage.java | 2 +- .../DistributedMetaStorageCasMessage.java | 4 +- ...istributedMetaStorageUpdateAckMessage.java | 2 +- .../DistributedMetaStorageUpdateMessage.java | 4 +- .../service/ServiceChangeBatchRequest.java | 4 +- .../ServiceClusterDeploymentResult.java | 4 +- .../ServiceClusterDeploymentResultBatch.java | 4 +- .../service/ServiceDeploymentRequest.java | 4 +- .../service/ServiceUndeploymentRequest.java | 4 +- .../util/distributed/FullMessage.java | 4 +- .../util/distributed/InitMessage.java | 4 +- .../util/distributed/SingleNodeMessage.java | 4 +- .../ignite/lang/IgniteProductVersion.java | 2 +- .../communication/MessageFactoryProvider.java | 2 +- .../tcp/internal/GridNioServerWrapper.java | 2 +- .../TcpConnectionRequestDiscoveryMessage.java | 4 +- .../tcp/internal/TcpHandshakeExecutor.java | 2 +- .../tcp/messages/InetAddressMessage.java | 4 +- .../messages/InetSocketAddressMessage.java | 4 +- .../TcpDiscoveryAbstractTraceableMessage.java | 4 +- .../TcpDiscoveryAuthFailedMessage.java | 4 +- ...cpDiscoveryClientMetricsUpdateMessage.java | 4 +- ...TcpDiscoveryClientNodesMetricsMessage.java | 4 +- .../TcpDiscoveryClientReconnectMessage.java | 4 +- .../TcpDiscoveryCollectionMessage.java | 4 +- .../TcpDiscoveryConnectionCheckMessage.java | 4 +- .../TcpDiscoveryCustomEventMessage.java | 6 +- .../TcpDiscoveryHandshakeRequest.java | 4 +- .../TcpDiscoveryHandshakeResponse.java | 4 +- .../TcpDiscoveryMetricsUpdateMessage.java | 4 +- .../TcpDiscoveryNodeAddedMessage.java | 4 +- .../messages/TcpDiscoveryNodeLeftMessage.java | 4 +- .../TcpDiscoveryRingLatencyCheckMessage.java | 4 +- .../TcpDiscoveryStatusCheckMessage.java | 4 +- .../direct/DirectMarshallingMessagesTest.java | 2 +- .../direct/TestNestedContainersMessage.java | 3 +- .../communication/CompressedMessageTest.java | 2 +- ...CommunicationMessageSerializationTest.java | 2 +- .../IgniteMessageFactoryImplTest.java | 2 +- .../MessageDirectTypeIdConflictTest.java | 2 +- ...niteDiscoveryMessageSerializationTest.java | 2 +- ...acheContinuousQueryImmutableEntryTest.java | 2 +- .../util/distributed/TestIntegerMessage.java | 3 +- .../util/distributed/TestUuidMessage.java | 3 +- .../GridAbstractCommunicationSelfTest.java | 2 +- ...unicationSpiConcurrentConnectSelfTest.java | 2 +- ...GridTcpCommunicationSpiConfigSelfTest.java | 2 +- ...CommunicationSpiMultithreadedSelfTest.java | 2 +- ...cpCommunicationSpiRecoveryAckSelfTest.java | 2 +- ...idTcpCommunicationSpiRecoverySelfTest.java | 2 +- ...ationSpiSkipWaitHandshakeOnClientTest.java | 2 +- ...mmunicationRecoveryAckClosureSelfTest.java | 2 +- .../tcp/DummyCustomDiscoveryMessage.java | 4 +- .../discovery/tcp/TestTcpDiscoverySpi.java | 4 +- .../testframework/GridSpiTestContext.java | 2 +- .../zk/internal/DiscoveryMessageParser.java | 2 +- 88 files changed, 167 insertions(+), 189 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/{managers => }/CoreMessagesProvider.java (96%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java similarity index 96% rename from modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java rename to modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 49f35b6ade371..42917b41908f5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -15,22 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.internal.managers; +package org.apache.ignite.internal; import java.lang.reflect.Constructor; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.ExchangeInfo; -import org.apache.ignite.internal.GridJobCancelRequest; -import org.apache.ignite.internal.GridJobExecuteRequest; -import org.apache.ignite.internal.GridJobExecuteResponse; -import org.apache.ignite.internal.GridJobSiblingsRequest; -import org.apache.ignite.internal.GridJobSiblingsResponse; -import org.apache.ignite.internal.GridTaskCancelRequest; -import org.apache.ignite.internal.GridTaskSessionRequest; -import org.apache.ignite.internal.IgniteDiagnosticRequest; -import org.apache.ignite.internal.IgniteDiagnosticResponse; -import org.apache.ignite.internal.TxEntriesInfo; -import org.apache.ignite.internal.TxInfo; import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; @@ -185,6 +173,10 @@ import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage; import org.apache.ignite.internal.processors.continuous.GridContinuousMessage; +import org.apache.ignite.internal.processors.continuous.StartRequestData; +import org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2; import org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage; import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage; import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry; @@ -323,12 +315,10 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { // [-44, 0..2, 42, 200..204, 210, 302] - Use in tests. // [300..307, 350..352] - CalciteMessageFactory. // [-4..-22, -30..-35, -54..-57] - SQL - // [-42..-37] - former hadoop. - // [64..71] - former IGFS. // [5000 - 5500]: Utility messages. Most of them originally come from Discovery. - // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. msgIdx = 5000; + // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. factory.register(msgIdx++, CompressedMessage::new); register(factory, ErrorMessage.class); register(factory, InetSocketAddressMessage.class); @@ -344,20 +334,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { msgIdx = 5500; register(factory, TcpDiscoveryCollectionMessage.class); - // [5700 - 5800]: Schema operation messages. Most of them originally come from Discovery. + // [5700 - 5900]: TcpDiscoveryAbstractMessage. msgIdx = 5700; - register(factory, SchemaAlterTableAddColumnOperation.class); - register(factory, SchemaIndexCreateOperation.class); - register(factory, SchemaIndexDropOperation.class); - register(factory, SchemaAlterTableDropColumnOperation.class); - register(factory, SchemaAddQueryEntityOperation.class); - register(factory, QueryField.class); - register(factory, SchemaOperationStatusMessage.class); - register(factory, SchemaProposeDiscoveryMessage.class); - register(factory, SchemaFinishDiscoveryMessage.class); - - // [5900 - 6100]: TcpDiscoveryAbstractMessage. - msgIdx = 5900; register(factory, TcpDiscoveryCheckFailedMessage.class); register(factory, TcpDiscoveryPingRequest.class); register(factory, TcpDiscoveryPingResponse.class); @@ -376,7 +354,6 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, TcpDiscoveryClientAckResponse.class); register(factory, TcpDiscoveryNodeLeftMessage.class); register(factory, TcpDiscoveryNodeFailedMessage.class); - register(factory, TcpDiscoveryStatusCheckMessage.class); register(factory, TcpDiscoveryNodeAddFinishedMessage.class); register(factory, TcpDiscoveryJoinRequestMessage.class); register(factory, TcpDiscoveryCustomEventMessage.class); @@ -384,8 +361,11 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, TcpDiscoveryNodeAddedMessage.class); register(factory, TcpDiscoveryClientReconnectMessage.class); - // [6200 - 6300]: Snapshot operation messages. Most of them originally come from Discovery. - msgIdx = 6200; + msgIdx = 5900; + register(factory, TcpDiscoveryStatusCheckMessage.class); + + // [6000 - 6200]: Snapshot operation messages. Most of them originally come from Discovery. + msgIdx = 6000; register(factory, SnapshotStartDiscoveryMessage.class); register(factory, SnapshotCheckProcessRequest.class); register(factory, SnapshotOperationRequest.class); @@ -404,8 +384,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, IncrementalSnapshotVerifyResult.class); register(factory, IncrementalSnapshotAwareMessage.class); - // [6400 - 6500]: Services messages. Most of them originally come from Discovery. - msgIdx = 6400; + // [6300 - 6400]: Services messages. Most of them originally come from Discovery. + msgIdx = 6300; register(factory, ServiceDeploymentProcessId.class); register(factory, ServiceSingleNodeDeploymentResult.class); register(factory, ServiceClusterDeploymentResult.class); @@ -415,12 +395,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, ServiceChangeBatchRequest.class); register(factory, ServiceSingleNodeDeploymentResultBatch.class); - // [6600 - 6700]: Originally Discovery's messages. - msgIdx = 6600; - - - // [6800 - 7000]: DiscoveryCustomMessage - msgIdx = 6800; + // [6500 - 6700]: DiscoveryCustomMessage + msgIdx = 6500; register(factory, TcpConnectionRequestDiscoveryMessage.class); register(factory, DistributedMetaStorageUpdateMessage.class); register(factory, DistributedMetaStorageUpdateAckMessage.class); @@ -518,6 +494,10 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, UpdateErrors.class); register(factory, LatchAckMessage.class); register(factory, AtomicApplicationAttributesAwareRequest.class); + register(factory, StartRequestData.class); + register(factory, StartRoutineDiscoveryMessage.class); + register(factory, StartRoutineAckDiscoveryMessage.class); + register(factory, StartRoutineDiscoveryMessageV2.class); // [10600-10800]: Affinity & partition maps. msgIdx = 10600; @@ -542,8 +522,17 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, GridDhtPartitionsSingleMessage.class); register(factory, GridDhtPartitionsSingleRequest.class); - // [10900-11100]: Query and SQL related messages. + // [10900-11100]: Query, schema and SQL related messages. msgIdx = 10900; + register(factory, SchemaAlterTableAddColumnOperation.class); + register(factory, SchemaIndexCreateOperation.class); + register(factory, SchemaIndexDropOperation.class); + register(factory, SchemaAlterTableDropColumnOperation.class); + register(factory, SchemaAddQueryEntityOperation.class); + register(factory, SchemaOperationStatusMessage.class); + register(factory, SchemaProposeDiscoveryMessage.class); + register(factory, SchemaFinishDiscoveryMessage.class); + register(factory, QueryField.class); register(factory, GridCacheSqlQuery.class); register(factory, GridCacheQueryRequest.class); register(factory, GridCacheQueryResponse.class); @@ -565,7 +554,7 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, CacheContinuousQueryBatchAck.class); register(factory, CacheContinuousQueryEntry.class); - // [11200 - 11300]: Compute and distributed process messages. + // [11200 - 11300]: Compute, distributed process messages. msgIdx = 11200; register(factory, GridJobCancelRequest.class); register(factory, GridJobExecuteRequest.class); @@ -585,12 +574,12 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, HandshakeMessage.class); register(factory, HandshakeWaitMessage.class); register(factory, GridIoMessage.class); + factory.register(msgIdx++, IgniteIoTestMessage::new); register(factory, GridIoUserMessage.class); register(factory, GridIoSecurityAwareMessage.class); register(factory, RecoveryLastReceivedMessage.class); register(factory, TcpInverseConnectionResponseMessage.class); register(factory, SessionChannelMessage.class); - factory.register(msgIdx++, IgniteIoTestMessage::new); // [11700 - 11800]: Datastreamer messages. msgIdx = 11700; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java index 5fc106e390754..aed8e35c29bc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Objects; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java index b0a67c4bf02e8..57a240b11fbe7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java @@ -24,7 +24,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java index eaca57f88101b..f742591f742b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index f81db699f8b02..2dd8268f0dd10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -94,7 +94,6 @@ import org.apache.ignite.internal.cluster.IgniteClusterEx; import org.apache.ignite.internal.maintenance.MaintenanceProcessor; import org.apache.ignite.internal.management.IgniteCommandRegistry; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.IgniteMBeansManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java index b14a39411815a..a5a9fcc6bc2f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java @@ -21,7 +21,6 @@ import java.util.HashSet; import java.util.Objects; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.KeyCacheObject; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java index 89ab35a2d0c72..16ad78229c4d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal; import java.util.Objects; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java index b40217bf0dcd5..b41d56110ce21 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java @@ -20,11 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.Nullable; @@ -50,7 +50,7 @@ public class SecurityAwareCustomMessageWrapper implements DiscoverySpiCustomMess @Order(2) byte[] msgBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SecurityAwareCustomMessageWrapper() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java index 549c813df5ae2..00b00bf7e65df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java @@ -20,8 +20,8 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Change cache group encryption key request. @@ -48,7 +48,7 @@ public class ChangeCacheEncryptionRequest implements Message { @Order(4) byte[] masterKeyDigest; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ChangeCacheEncryptionRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java index 9ad784f7b3fb7..a6a85dfee4822 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java @@ -21,9 +21,9 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Master key change request. */ public class MasterKeyChangeRequest implements Message { @@ -39,7 +39,7 @@ public class MasterKeyChangeRequest implements Message { @Order(2) byte[] digest; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public MasterKeyChangeRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java index 0e38787604a08..02edc0d90aa9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java @@ -21,11 +21,11 @@ import java.util.Collections; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -58,7 +58,7 @@ public class CacheStatisticsModeChangeMessage implements DiscoveryCustomMessage, byte flags; /** - * Constructor for {@link CoreMessagesProvider}. + * Constructor for {@link MessageFactory}. */ public CacheStatisticsModeChangeMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java index 720f57c44890a..e46e0cf6dcf5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java @@ -25,7 +25,6 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; @@ -38,6 +37,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -68,7 +68,7 @@ public class ExchangeFailureMessage implements DiscoveryCustomMessage, Message { /** Actions to be done to rollback changes done before the exchange failure. */ private transient ExchangeActions exchangeRollbackActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ExchangeFailureMessage() { // No-op. } 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 7d47676273d35..acd4baab4f86d 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 @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.distributed; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java index fc3f766a0c6fb..a12210c29cb77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java @@ -22,11 +22,11 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -59,7 +59,7 @@ abstract class AbstractSnapshotOperationRequest implements Message { @Order(5) Set nodes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public AbstractSnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java index c16735839ff45..ffcf11b7a1a35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class DataStreamerUpdatesHandlerResult implements Message { @@ -27,7 +27,7 @@ public class DataStreamerUpdatesHandlerResult implements Message { @Order(0) boolean streamerWarning; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public DataStreamerUpdatesHandlerResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java index 88215dce9104c..d614f2071d303 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java @@ -20,7 +20,6 @@ import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; @@ -30,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @@ -58,7 +58,7 @@ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @Order(3) Collection exceptions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public IncrementalSnapshotVerifyResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java index 6ae8ff9aef638..41ab204b19355 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java @@ -19,8 +19,8 @@ import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckHandlersResponse implements Message { @@ -28,7 +28,7 @@ public class SnapshotCheckHandlersResponse implements Message { @Order(0) Map>> perMetaResults; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckHandlersResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java index 504a437906928..f5489867ffcfe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage @Order(0) byte[] perMetaResultsBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckPartitionHashesResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java index 5e4900a8fc65c..791a8f559694b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java @@ -20,10 +20,10 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques @Order(2) int incIdx; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckProcessRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java index fd6ea067aee39..693576027fd84 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java @@ -21,10 +21,10 @@ import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** A DTO to transfer node's results for the both phases. */ @@ -37,7 +37,7 @@ public final class SnapshotCheckResponse implements Message { @Order(1) @Nullable Map errors; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java index d70ac1f46ac6b..02b5bd9b31f2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java @@ -18,9 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -38,7 +38,7 @@ public class SnapshotHandlerResult implements Message { @Order(1) ErrorMessage errMsg; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotHandlerResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java index 33b8f6dc3ec81..d45325ad093ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java @@ -20,10 +20,10 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotMetadataResponse implements MarshallableMessage { @@ -34,7 +34,7 @@ public class SnapshotMetadataResponse implements MarshallableMessage { /** */ private List metadata; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotMetadataResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java index ed3d0dacb8432..3b24db0c1a8e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java @@ -20,12 +20,12 @@ import java.util.List; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -48,7 +48,7 @@ public class SnapshotOperationEndRequest implements Message { @Order(2) @Nullable List warnings; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationEndRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index bb93b19176809..25aa5fb10a57d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -21,9 +21,9 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class SnapshotOperationRequest extends AbstractSnapshotOperationRequest { @Order(7) boolean configOnly; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java index e9b1f434c70fd..5f023f2e3aedd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java @@ -19,8 +19,8 @@ import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -29,7 +29,7 @@ public class SnapshotOperationResponse implements Message { @Order(0) Map> hndResults; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java index 0435d470e0a89..40ea2671f3946 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMess @Order(0) byte[] resBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotPartitionsVerifyHandlerResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java index b9426cfb36f7e..2f658cf168073 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java @@ -22,11 +22,11 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Snapshot operation prepare response. */ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @@ -44,7 +44,7 @@ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @Order(1) byte[] metasBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotRestoreOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java index 865f954b5f1ab..5f8b82713243c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotRestoreStartRequest implements Message { @@ -29,7 +29,7 @@ public class SnapshotRestoreStartRequest implements Message { @Order(0) UUID reqId; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotRestoreStartRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java index 23fce6364f11e..c7611c0b0bee8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.InitMessage; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT; @@ -34,7 +34,7 @@ public class SnapshotStartDiscoveryMessage extends InitMessage allNodesMetrics; - /** Default constructor. Required for {@link CoreMessagesProvider}. */ + /** Default constructor. Required for {@link MessageFactory}. */ public ClusterMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java index 352ec9291cefc..b52a355b61083 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java @@ -21,10 +21,10 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Node compound metrics message. */ public class NodeFullMetricsMessage implements Message { @@ -36,7 +36,7 @@ public class NodeFullMetricsMessage implements Message { @Order(1) public Map cachesMetricsMsgs; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public NodeFullMetricsMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java index e6fd484913936..35bd6407a4b13 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.metastorage.persistence; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java index 117584b9fd0b9..f3817d43f175f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -37,7 +37,7 @@ public class DistributedMetaStorageCasMessage extends DistributedMetaStorageUpda @Order(1) boolean matches; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public DistributedMetaStorageCasMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java index dd410581ac2f4..181f3f35751b4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.metastorage.persistence; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java index 2127197afe69a..dabd0a791354b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java @@ -19,12 +19,12 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -50,7 +50,7 @@ public class DistributedMetaStorageUpdateMessage implements DiscoveryCustomMessa @Order(3) byte[] valBytes; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public DistributedMetaStorageUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java index e9965f22b4d73..875432ccbfbbd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java @@ -20,7 +20,6 @@ import java.util.Collection; import java.util.Collections; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -28,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class ServiceChangeBatchRequest implements DiscoveryCustomMessage, Messag @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceChangeBatchRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java index 5d6b775125fab..655f70dfd45ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java @@ -21,11 +21,11 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -43,7 +43,7 @@ public class ServiceClusterDeploymentResult implements Message { @GridToStringInclude Map results; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceClusterDeploymentResult() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java index 20c4634047e91..f947d54131eec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java @@ -19,13 +19,13 @@ import java.util.Collection; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -55,7 +55,7 @@ public class ServiceClusterDeploymentResultBatch implements DiscoveryCustomMessa @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceClusterDeploymentResultBatch() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java index a243275bd04a5..4d1813477ad91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java @@ -19,12 +19,12 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -38,7 +38,7 @@ public class ServiceDeploymentRequest extends ServiceChangeAbstractRequest imple @Order(0) byte[] cfgBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceDeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java index 99a85aef7cb9c..5a5911c65332f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java @@ -17,16 +17,16 @@ package org.apache.ignite.internal.processors.service; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** * Service undeployment request. */ public class ServiceUndeploymentRequest extends ServiceChangeAbstractRequest { - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceUndeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java index 6ed2194d5a572..9639f4c843ec9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java @@ -21,7 +21,6 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; @@ -29,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -63,7 +63,7 @@ public class FullMessage implements DiscoveryCustomMessage, M @Order(4) Map err; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public FullMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java index b7aadc67bf089..cf9412cb587d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java @@ -19,7 +19,6 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -28,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class InitMessage implements Message, DiscoveryCustomM @Order(4) public boolean waitClnRes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public InitMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java index 38786258f74ba..ddd06005c3aeb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java @@ -19,10 +19,10 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class SingleNodeMessage implements Message { @Order(3) @Nullable ErrorMessage errMsg; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SingleNodeMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java index e66fcb41f34a0..b4177fcfd7810 100644 --- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java @@ -25,9 +25,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.NotNull; diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java index 5e36086bc2d9d..df25d13a17bd6 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java @@ -17,7 +17,7 @@ package org.apache.ignite.plugin.extensions.communication; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.Extension; /** diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java index 6735edf6602e4..b8eee2efbc59a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java @@ -48,12 +48,12 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteTooManyOpenFilesException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.GridManager; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.tracing.GridTracingManager; import org.apache.ignite.internal.processors.metric.GridMetricManager; import org.apache.ignite.internal.processors.tracing.Tracing; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java index bf82966e14279..41583a1a3564c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java @@ -19,12 +19,12 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -63,7 +63,7 @@ public TcpConnectionRequestDiscoveryMessage(UUID receiverNodeId, int connIdx) { this.connIdx = connIdx; } - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpConnectionRequestDiscoveryMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java index 4258926d5470b..7a609d2f7ebdf 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java @@ -25,7 +25,7 @@ import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; import org.apache.ignite.internal.util.nio.ssl.GridSslMeta; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java index 3d905dad3a536..8b62cf7adabf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java @@ -21,9 +21,9 @@ import java.net.UnknownHostException; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Address utility container message. Is not a pure {@link TcpDiscoveryAbstractMessage}. */ public class InetAddressMessage implements Message { @@ -35,7 +35,7 @@ public class InetAddressMessage implements Message { @Order(1) byte[] addrBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public InetAddressMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java index e792b9b21e8f5..f23e36f200d27 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java @@ -19,8 +19,8 @@ import java.net.InetAddress; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Socket address utility container message. Is not a pure {@link TcpDiscoveryAbstractMessage}. */ public class InetSocketAddressMessage extends InetAddressMessage { @@ -29,7 +29,7 @@ public class InetSocketAddressMessage extends InetAddressMessage { int port; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public InetSocketAddressMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java index 01d937cde8028..2df35f55ce64a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java @@ -20,11 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.tracing.messages.SpanContainer; import org.apache.ignite.internal.processors.tracing.messages.TraceableMessage; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -40,7 +40,7 @@ public abstract class TcpDiscoveryAbstractTraceableMessage extends TcpDiscoveryA @Nullable byte[] spanContainerBytes; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ protected TcpDiscoveryAbstractTraceableMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java index c7ee4aaececc4..bf3a97da951b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java @@ -20,9 +20,9 @@ import java.net.InetAddress; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Message telling joining node that its authentication failed. @@ -39,7 +39,7 @@ public class TcpDiscoveryAuthFailedMessage extends TcpDiscoveryAbstractMessage i @Order(1) UUID targetNodeId; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TcpDiscoveryAuthFailedMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java index 515e07397fe63..8851ae20bb478 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java @@ -20,10 +20,10 @@ import java.util.UUID; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Metrics update message. @@ -38,7 +38,7 @@ public class TcpDiscoveryClientMetricsUpdateMessage extends TcpDiscoveryAbstract @Order(0) NodeMetricsMessage metricsMsg; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java index 140d3b9977b5b..7aa4afa0b0be8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java @@ -20,10 +20,10 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Holds map of thick client or server metrics messages per node id. */ public class TcpDiscoveryClientNodesMetricsMessage implements Message { @@ -31,7 +31,7 @@ public class TcpDiscoveryClientNodesMetricsMessage implements Message { @Order(0) Map nodesMetricsMsgs; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientNodesMetricsMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java index ffda34cf60c00..6aa882964358d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java @@ -22,11 +22,11 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -49,7 +49,7 @@ public class TcpDiscoveryClientReconnectMessage extends TcpDiscoveryAbstractMess @Order(2) @Nullable TcpDiscoveryCollectionMessage pendingMsgsMsg; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientReconnectMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java index eea757b4a34a4..0d3c2b864aee6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java @@ -24,7 +24,6 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -32,6 +31,7 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -52,7 +52,7 @@ public class TcpDiscoveryCollectionMessage implements MarshallableMessage { @GridToStringExclude @Nullable byte[] marshallableMsgsBytes; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryCollectionMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java index dbea16218c2eb..2f938710cd091 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java @@ -17,9 +17,9 @@ package org.apache.ignite.spi.discovery.tcp.messages; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; /** @@ -32,7 +32,7 @@ public class TcpDiscoveryConnectionCheckMessage extends TcpDiscoveryAbstractMess private static final long serialVersionUID = 0L; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryConnectionCheckMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java index b134410faee5c..1e2b713998c75 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java @@ -21,13 +21,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; -import org.apache.ignite.internal.managers.CoreMessagesProvider; -import org.apache.ignite.internal.managers.discovery.IncompleteDeserializationException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.Nullable; @@ -54,7 +52,7 @@ public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractTraceabl volatile @Nullable Message serMsg; /** - * Constructor for {@link CoreMessagesProvider}. + * Constructor for {@link MessageFactory}. */ public TcpDiscoveryCustomEventMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java index 9e503b87594af..d553cf5910976 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -40,7 +40,7 @@ public class TcpDiscoveryHandshakeRequest extends TcpDiscoveryAbstractMessage im @Nullable String dcId; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryHandshakeRequest() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java index 21f207e6fc6b3..e3381ab5aaeb4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java @@ -21,10 +21,10 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -47,7 +47,7 @@ public class TcpDiscoveryHandshakeResponse extends TcpDiscoveryAbstractMessage i @Nullable Collection redirectAddrsMsgs; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryHandshakeResponse() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java index 1f4213a92191b..522f66e5dfbe8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java @@ -25,7 +25,6 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; @@ -33,6 +32,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -68,7 +68,7 @@ public class TcpDiscoveryMetricsUpdateMessage extends TcpDiscoveryAbstractMessag @Order(2) @Nullable Set clientNodeIds; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java index 4b4b148af9091..e47dac2443c13 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java @@ -21,10 +21,10 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.jetbrains.annotations.Nullable; @@ -69,7 +69,7 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractTraceableM @Order(5) long gridStartTime; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryNodeAddedMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java index 9e50d472ddfce..9f6071d629af0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.spi.discovery.tcp.messages; import java.util.UUID; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Sent by node that is stopping to coordinator across the ring, @@ -31,7 +31,7 @@ public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractTraceableMe /** */ private static final long serialVersionUID = 0L; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryNodeLeftMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java index 6067262b4d0eb..6e07a1167a072 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * @@ -38,7 +38,7 @@ public class TcpDiscoveryRingLatencyCheckMessage extends TcpDiscoveryAbstractMes @Order(1) int curHops; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public TcpDiscoveryRingLatencyCheckMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java index 14fe94552ab4b..c9d8385b2560f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java @@ -23,9 +23,9 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -58,7 +58,7 @@ public class TcpDiscoveryStatusCheckMessage extends TcpDiscoveryAbstractMessage @Order(2) int status; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public TcpDiscoveryStatusCheckMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java index 30c44be358a39..a4c9164388911 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java index 5f1a56622ede0..32ef435f32077 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -40,7 +39,7 @@ class TestNestedContainersMessage implements Message { @Order(2) Map nestedArr; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestNestedContainersMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java index b146af2c53967..93e846ce703b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java @@ -22,11 +22,11 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.direct.state.DirectMessageState; import org.apache.ignite.internal.direct.stream.DirectByteBufferStream; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPair; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java index e5be7dd50c8d2..8c9a0e2b8e19d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java @@ -19,7 +19,7 @@ import java.util.UUID; import org.apache.commons.lang3.reflect.FieldUtils; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java index 53a4cad334664..c34eb96cf4d02 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java index 4197222ab2caf..a9df4dbc68678 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java @@ -21,7 +21,7 @@ import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.AbstractTestPluginProvider; import org.apache.ignite.plugin.ExtensionRegistry; import org.apache.ignite.plugin.PluginContext; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java index 4dc2ddedd0b34..c3f51d523a05a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.managers.discovery; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.AbstractMessageSerializationTest; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java index 2473dacf942e5..15ddd54040c5a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java @@ -30,9 +30,9 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java index b960d0f886dbe..da1b7ca67fc8f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.util.distributed; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -27,7 +26,7 @@ public class TestIntegerMessage implements Message { @Order(0) int val; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestIntegerMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java index 4dc4ebee4bd14..57940c1e17ffe 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java @@ -19,7 +19,6 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -28,7 +27,7 @@ public class TestUuidMessage implements Message { @Order(0) UUID val; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestUuidMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index 10615b39022a4..f82c844bda973 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 31a4f880344b8..1bc2358cc7e2f 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -35,9 +35,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteNodeAttributes; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java index 4f448c0f3c1b5..a604da0edb7f0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java @@ -25,9 +25,9 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 62fcd6d55569c..6a5d26eb26619 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -36,9 +36,9 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.nio.GridCommunicationClient; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index 648d7c3febb5c..de46f4b674a88 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index 9e1c6145459b7..633bd332eaf28 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -30,9 +30,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java index 70fa1b6c967b3..d5add388704b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java @@ -20,8 +20,8 @@ import java.io.InputStream; import java.net.Socket; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.ListeningTestLogger; import org.apache.ignite.testframework.LogListener; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index 71aaae35c5f38..06d657a8a3bea 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java index 8eeae66fac4e5..a26013a586ff4 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java @@ -19,9 +19,9 @@ import org.apache.ignite.internal.Order; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -30,7 +30,7 @@ public class DummyCustomDiscoveryMessage implements DiscoveryCustomMessage, Mess @Order(0) IgniteUuid id = IgniteUuid.randomUuid(); - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link MessageFactory}. */ public DummyCustomDiscoveryMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java index 859c0979beb61..decf2bf4b5c3e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java @@ -20,8 +20,8 @@ import java.io.IOException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageFactory; @@ -121,7 +121,7 @@ public void messageFactory(MessageFactoryProvider msgFactoryProvider) { assert !started(); this.msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new DiscoveryMessageFactory(jdk(), U.resolveClassLoader(ignite().configuration())), + new CoreMessagesProvider(jdk(), U.resolveClassLoader(ignite().configuration())), msgFactoryProvider }); } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index 62391811edc5e..ce5ac8e79f69f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -37,10 +37,10 @@ import org.apache.ignite.events.Event; import org.apache.ignite.events.TaskEvent; import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridIoUserMessage; import org.apache.ignite.internal.managers.communication.GridMessageListener; diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java index 77b75d6922694..fec34ef4a336d 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java @@ -26,9 +26,9 @@ import java.nio.ByteBuffer; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; From ef86ca4c3a14bb7c7b5c61b4a11dcc67c5d2366e Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 14:17:58 +0300 Subject: [PATCH 3/9] + master, checkstyle, cleanup --- .../{managers => }/CoreMessagesProvider.java | 107 ++++++++---------- .../apache/ignite/internal/ExchangeInfo.java | 1 - .../internal/IgniteDiagnosticRequest.java | 1 - .../internal/IgniteDiagnosticResponse.java | 1 - .../apache/ignite/internal/IgniteKernal.java | 1 - .../apache/ignite/internal/TxEntriesInfo.java | 1 - .../org/apache/ignite/internal/TxInfo.java | 1 - .../SecurityAwareCustomMessageWrapper.java | 4 +- .../ChangeCacheEncryptionRequest.java | 4 +- .../encryption/MasterKeyChangeRequest.java | 4 +- .../CacheStatisticsModeChangeMessage.java | 4 +- .../cache/ExchangeFailureMessage.java | 4 +- .../GridDistributedTxFinishResponse.java | 2 +- .../AbstractSnapshotOperationRequest.java | 4 +- .../DataStreamerUpdatesHandlerResult.java | 4 +- .../IncrementalSnapshotVerifyResult.java | 4 +- .../SnapshotCheckHandlersResponse.java | 4 +- .../SnapshotCheckPartitionHashesResponse.java | 4 +- .../snapshot/SnapshotCheckProcessRequest.java | 4 +- .../snapshot/SnapshotCheckResponse.java | 4 +- .../snapshot/SnapshotHandlerResult.java | 4 +- .../snapshot/SnapshotMetadataResponse.java | 4 +- .../snapshot/SnapshotOperationEndRequest.java | 4 +- .../snapshot/SnapshotOperationRequest.java | 4 +- .../snapshot/SnapshotOperationResponse.java | 4 +- ...apshotPartitionsVerifyHandlerResponse.java | 4 +- .../SnapshotRestoreOperationResponse.java | 4 +- .../snapshot/SnapshotRestoreStartRequest.java | 4 +- .../SnapshotStartDiscoveryMessage.java | 4 +- .../cluster/CacheMetricsMessage.java | 4 +- .../cluster/ClusterMetricsUpdateMessage.java | 4 +- .../cluster/NodeFullMetricsMessage.java | 4 +- .../DistributedMetaStorageCasAckMessage.java | 2 +- .../DistributedMetaStorageCasMessage.java | 4 +- ...istributedMetaStorageUpdateAckMessage.java | 2 +- .../DistributedMetaStorageUpdateMessage.java | 4 +- .../service/ServiceChangeBatchRequest.java | 4 +- .../ServiceClusterDeploymentResult.java | 4 +- .../ServiceClusterDeploymentResultBatch.java | 4 +- .../service/ServiceDeploymentRequest.java | 4 +- .../service/ServiceUndeploymentRequest.java | 4 +- .../util/distributed/FullMessage.java | 4 +- .../util/distributed/InitMessage.java | 4 +- .../util/distributed/SingleNodeMessage.java | 4 +- .../ignite/lang/IgniteProductVersion.java | 2 +- .../communication/MessageFactoryProvider.java | 2 +- .../tcp/internal/GridNioServerWrapper.java | 2 +- .../TcpConnectionRequestDiscoveryMessage.java | 4 +- .../tcp/internal/TcpHandshakeExecutor.java | 2 +- .../tcp/messages/InetAddressMessage.java | 4 +- .../messages/InetSocketAddressMessage.java | 4 +- .../TcpDiscoveryAbstractTraceableMessage.java | 4 +- .../TcpDiscoveryAuthFailedMessage.java | 4 +- ...cpDiscoveryClientMetricsUpdateMessage.java | 4 +- ...TcpDiscoveryClientNodesMetricsMessage.java | 4 +- .../TcpDiscoveryClientReconnectMessage.java | 4 +- .../TcpDiscoveryCollectionMessage.java | 4 +- .../TcpDiscoveryConnectionCheckMessage.java | 4 +- .../TcpDiscoveryCustomEventMessage.java | 6 +- .../TcpDiscoveryHandshakeRequest.java | 4 +- .../TcpDiscoveryHandshakeResponse.java | 4 +- .../TcpDiscoveryMetricsUpdateMessage.java | 4 +- .../TcpDiscoveryNodeAddedMessage.java | 4 +- .../messages/TcpDiscoveryNodeLeftMessage.java | 4 +- .../TcpDiscoveryRingLatencyCheckMessage.java | 4 +- .../TcpDiscoveryStatusCheckMessage.java | 4 +- .../direct/DirectMarshallingMessagesTest.java | 2 +- .../direct/TestNestedContainersMessage.java | 3 +- .../communication/CompressedMessageTest.java | 2 +- ...CommunicationMessageSerializationTest.java | 2 +- .../IgniteMessageFactoryImplTest.java | 2 +- .../MessageDirectTypeIdConflictTest.java | 2 +- ...niteDiscoveryMessageSerializationTest.java | 2 +- ...acheContinuousQueryImmutableEntryTest.java | 2 +- .../util/distributed/TestIntegerMessage.java | 3 +- .../util/distributed/TestUuidMessage.java | 3 +- .../GridAbstractCommunicationSelfTest.java | 2 +- ...unicationSpiConcurrentConnectSelfTest.java | 2 +- ...GridTcpCommunicationSpiConfigSelfTest.java | 2 +- ...CommunicationSpiMultithreadedSelfTest.java | 2 +- ...cpCommunicationSpiRecoveryAckSelfTest.java | 2 +- ...idTcpCommunicationSpiRecoverySelfTest.java | 2 +- ...ationSpiSkipWaitHandshakeOnClientTest.java | 2 +- ...mmunicationRecoveryAckClosureSelfTest.java | 2 +- .../tcp/DummyCustomDiscoveryMessage.java | 4 +- .../discovery/tcp/TestTcpDiscoverySpi.java | 4 +- .../testframework/GridSpiTestContext.java | 2 +- .../zk/internal/DiscoveryMessageParser.java | 2 +- 88 files changed, 183 insertions(+), 205 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/{managers => }/CoreMessagesProvider.java (96%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java similarity index 96% rename from modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java rename to modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 49f35b6ade371..82285e93e50cb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -15,22 +15,10 @@ * limitations under the License. */ -package org.apache.ignite.internal.managers; +package org.apache.ignite.internal; import java.lang.reflect.Constructor; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.ExchangeInfo; -import org.apache.ignite.internal.GridJobCancelRequest; -import org.apache.ignite.internal.GridJobExecuteRequest; -import org.apache.ignite.internal.GridJobExecuteResponse; -import org.apache.ignite.internal.GridJobSiblingsRequest; -import org.apache.ignite.internal.GridJobSiblingsResponse; -import org.apache.ignite.internal.GridTaskCancelRequest; -import org.apache.ignite.internal.GridTaskSessionRequest; -import org.apache.ignite.internal.IgniteDiagnosticRequest; -import org.apache.ignite.internal.IgniteDiagnosticResponse; -import org.apache.ignite.internal.TxEntriesInfo; -import org.apache.ignite.internal.TxInfo; import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; @@ -185,6 +173,10 @@ import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.processors.continuous.ContinuousRoutineStartResultMessage; import org.apache.ignite.internal.processors.continuous.GridContinuousMessage; +import org.apache.ignite.internal.processors.continuous.StartRequestData; +import org.apache.ignite.internal.processors.continuous.StartRoutineAckDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessage; +import org.apache.ignite.internal.processors.continuous.StartRoutineDiscoveryMessageV2; import org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage; import org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage; import org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry; @@ -323,12 +315,10 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { // [-44, 0..2, 42, 200..204, 210, 302] - Use in tests. // [300..307, 350..352] - CalciteMessageFactory. // [-4..-22, -30..-35, -54..-57] - SQL - // [-42..-37] - former hadoop. - // [64..71] - former IGFS. // [5000 - 5500]: Utility messages. Most of them originally come from Discovery. - // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. msgIdx = 5000; + // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. factory.register(msgIdx++, CompressedMessage::new); register(factory, ErrorMessage.class); register(factory, InetSocketAddressMessage.class); @@ -344,48 +334,35 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { msgIdx = 5500; register(factory, TcpDiscoveryCollectionMessage.class); - // [5700 - 5800]: Schema operation messages. Most of them originally come from Discovery. + // [5700 - 5900]: Discovery originated messages. msgIdx = 5700; - register(factory, SchemaAlterTableAddColumnOperation.class); - register(factory, SchemaIndexCreateOperation.class); - register(factory, SchemaIndexDropOperation.class); - register(factory, SchemaAlterTableDropColumnOperation.class); - register(factory, SchemaAddQueryEntityOperation.class); - register(factory, QueryField.class); - register(factory, SchemaOperationStatusMessage.class); - register(factory, SchemaProposeDiscoveryMessage.class); - register(factory, SchemaFinishDiscoveryMessage.class); - - // [5900 - 6100]: TcpDiscoveryAbstractMessage. - msgIdx = 5900; - register(factory, TcpDiscoveryCheckFailedMessage.class); + register(factory, TcpDiscoveryHandshakeRequest.class); + register(factory, TcpDiscoveryHandshakeResponse.class); + register(factory, TcpDiscoveryJoinRequestMessage.class); + register(factory, TcpDiscoveryNodeAddedMessage.class); + register(factory, TcpDiscoveryNodeAddFinishedMessage.class); + register(factory, TcpDiscoveryNodeLeftMessage.class); + register(factory, TcpDiscoveryNodeFailedMessage.class); + register(factory, TcpDiscoveryConnectionCheckMessage.class); register(factory, TcpDiscoveryPingRequest.class); register(factory, TcpDiscoveryPingResponse.class); register(factory, TcpDiscoveryClientPingRequest.class); register(factory, TcpDiscoveryClientPingResponse.class); + register(factory, TcpDiscoveryClientAckResponse.class); + register(factory, TcpDiscoveryClientReconnectMessage.class); + register(factory, TcpDiscoveryDiscardMessage.class); + register(factory, TcpDiscoveryCheckFailedMessage.class); register(factory, TcpDiscoveryLoopbackProblemMessage.class); - register(factory, TcpDiscoveryConnectionCheckMessage.class); register(factory, TcpDiscoveryRingLatencyCheckMessage.class); - register(factory, TcpDiscoveryHandshakeRequest.class); - register(factory, TcpDiscoveryDiscardMessage.class); - register(factory, TcpDiscoveryHandshakeResponse.class); - register(factory, TcpDiscoveryAuthFailedMessage.class); register(factory, TcpDiscoveryDuplicateIdMessage.class); - register(factory, TcpDiscoveryClientMetricsUpdateMessage.class); - register(factory, TcpDiscoveryMetricsUpdateMessage.class); - register(factory, TcpDiscoveryClientAckResponse.class); - register(factory, TcpDiscoveryNodeLeftMessage.class); - register(factory, TcpDiscoveryNodeFailedMessage.class); - register(factory, TcpDiscoveryStatusCheckMessage.class); - register(factory, TcpDiscoveryNodeAddFinishedMessage.class); - register(factory, TcpDiscoveryJoinRequestMessage.class); register(factory, TcpDiscoveryCustomEventMessage.class); register(factory, TcpDiscoveryServerOnlyCustomEventMessage.class); - register(factory, TcpDiscoveryNodeAddedMessage.class); - register(factory, TcpDiscoveryClientReconnectMessage.class); - // [6200 - 6300]: Snapshot operation messages. Most of them originally come from Discovery. - msgIdx = 6200; + msgIdx = 5900; + register(factory, TcpDiscoveryStatusCheckMessage.class); + + // [6000 - 6200]: Snapshot operation messages. Most of them originally come from Discovery. + msgIdx = 6000; register(factory, SnapshotStartDiscoveryMessage.class); register(factory, SnapshotCheckProcessRequest.class); register(factory, SnapshotOperationRequest.class); @@ -404,8 +381,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, IncrementalSnapshotVerifyResult.class); register(factory, IncrementalSnapshotAwareMessage.class); - // [6400 - 6500]: Services messages. Most of them originally come from Discovery. - msgIdx = 6400; + // [6300 - 6400]: Services messages. Most of them originally come from Discovery. + msgIdx = 6300; register(factory, ServiceDeploymentProcessId.class); register(factory, ServiceSingleNodeDeploymentResult.class); register(factory, ServiceClusterDeploymentResult.class); @@ -415,12 +392,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, ServiceChangeBatchRequest.class); register(factory, ServiceSingleNodeDeploymentResultBatch.class); - // [6600 - 6700]: Originally Discovery's messages. - msgIdx = 6600; - - - // [6800 - 7000]: DiscoveryCustomMessage - msgIdx = 6800; + // [6500 - 6700]: DiscoveryCustomMessage + msgIdx = 6500; register(factory, TcpConnectionRequestDiscoveryMessage.class); register(factory, DistributedMetaStorageUpdateMessage.class); register(factory, DistributedMetaStorageUpdateAckMessage.class); @@ -447,7 +420,6 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, MappingProposedMessage.class); register(factory, ExchangeFailureMessage.class); register(factory, CacheStatisticsClearMessage.class); - register(factory, ChangeGlobalStateMessage.class); register(factory, ClientCacheChangeDummyDiscoveryMessage.class); register(factory, DynamicCacheChangeBatch.class); @@ -518,6 +490,10 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, UpdateErrors.class); register(factory, LatchAckMessage.class); register(factory, AtomicApplicationAttributesAwareRequest.class); + register(factory, StartRequestData.class); + register(factory, StartRoutineDiscoveryMessage.class); + register(factory, StartRoutineAckDiscoveryMessage.class); + register(factory, StartRoutineDiscoveryMessageV2.class); // [10600-10800]: Affinity & partition maps. msgIdx = 10600; @@ -542,8 +518,17 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, GridDhtPartitionsSingleMessage.class); register(factory, GridDhtPartitionsSingleRequest.class); - // [10900-11100]: Query and SQL related messages. + // [10900-11100]: Query, schema and SQL related messages. msgIdx = 10900; + register(factory, SchemaAlterTableAddColumnOperation.class); + register(factory, SchemaIndexCreateOperation.class); + register(factory, SchemaIndexDropOperation.class); + register(factory, SchemaAlterTableDropColumnOperation.class); + register(factory, SchemaAddQueryEntityOperation.class); + register(factory, SchemaOperationStatusMessage.class); + register(factory, SchemaProposeDiscoveryMessage.class); + register(factory, SchemaFinishDiscoveryMessage.class); + register(factory, QueryField.class); register(factory, GridCacheSqlQuery.class); register(factory, GridCacheQueryRequest.class); register(factory, GridCacheQueryResponse.class); @@ -565,7 +550,7 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, CacheContinuousQueryBatchAck.class); register(factory, CacheContinuousQueryEntry.class); - // [11200 - 11300]: Compute and distributed process messages. + // [11200 - 11300]: Compute, distributed process messages. msgIdx = 11200; register(factory, GridJobCancelRequest.class); register(factory, GridJobExecuteRequest.class); @@ -585,12 +570,12 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, HandshakeMessage.class); register(factory, HandshakeWaitMessage.class); register(factory, GridIoMessage.class); + factory.register(msgIdx++, IgniteIoTestMessage::new); register(factory, GridIoUserMessage.class); register(factory, GridIoSecurityAwareMessage.class); register(factory, RecoveryLastReceivedMessage.class); register(factory, TcpInverseConnectionResponseMessage.class); register(factory, SessionChannelMessage.class); - factory.register(msgIdx++, IgniteIoTestMessage::new); // [11700 - 11800]: Datastreamer messages. msgIdx = 11700; @@ -606,6 +591,8 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, NodeFullMetricsMessage.class); register(factory, ClusterMetricsUpdateMessage.class); register(factory, TcpDiscoveryClientNodesMetricsMessage.class); + register(factory, TcpDiscoveryMetricsUpdateMessage.class); + register(factory, TcpDiscoveryClientMetricsUpdateMessage.class); // [12000 - 12100]: Authentication, security messages. msgIdx = 12000; @@ -614,6 +601,7 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(factory, UserManagementOperationFinishedMessage.class); register(factory, UserAuthenticateRequestMessage.class); register(factory, UserAuthenticateResponseMessage.class); + register(factory, TcpDiscoveryAuthFailedMessage.class); // [12200 - 12300]: Binary, classloading and marshalling messages. msgIdx = 12200; @@ -637,6 +625,7 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { // [13000 - 13300]: Control, diagnostincs and other messages. msgIdx = 13000; register(factory, GridEventStorageMessage.class); + register(factory, ChangeGlobalStateMessage.class); register(factory, GridChangeGlobalStateMessageResponse.class); register(factory, IgniteDiagnosticRequest.class); register(factory, IgniteDiagnosticResponse.class); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java index 5fc106e390754..aed8e35c29bc7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/ExchangeInfo.java @@ -19,7 +19,6 @@ import java.util.List; import java.util.Objects; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java index b0a67c4bf02e8..57a240b11fbe7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticRequest.java @@ -24,7 +24,6 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java index eaca57f88101b..f742591f742b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteDiagnosticResponse.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.Nullable; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index f81db699f8b02..2dd8268f0dd10 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -94,7 +94,6 @@ import org.apache.ignite.internal.cluster.IgniteClusterEx; import org.apache.ignite.internal.maintenance.MaintenanceProcessor; import org.apache.ignite.internal.management.IgniteCommandRegistry; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.GridManager; import org.apache.ignite.internal.managers.IgniteMBeansManager; import org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java index b14a39411815a..a5a9fcc6bc2f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxEntriesInfo.java @@ -21,7 +21,6 @@ import java.util.HashSet; import java.util.Objects; import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheMapEntry; import org.apache.ignite.internal.processors.cache.KeyCacheObject; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java index 89ab35a2d0c72..16ad78229c4d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TxInfo.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal; import java.util.Objects; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java index b40217bf0dcd5..b41d56110ce21 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/SecurityAwareCustomMessageWrapper.java @@ -20,11 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.Nullable; @@ -50,7 +50,7 @@ public class SecurityAwareCustomMessageWrapper implements DiscoverySpiCustomMess @Order(2) byte[] msgBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SecurityAwareCustomMessageWrapper() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java index 549c813df5ae2..00b00bf7e65df 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/ChangeCacheEncryptionRequest.java @@ -20,8 +20,8 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Change cache group encryption key request. @@ -48,7 +48,7 @@ public class ChangeCacheEncryptionRequest implements Message { @Order(4) byte[] masterKeyDigest; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ChangeCacheEncryptionRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java index 9ad784f7b3fb7..a6a85dfee4822 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/encryption/MasterKeyChangeRequest.java @@ -21,9 +21,9 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Master key change request. */ public class MasterKeyChangeRequest implements Message { @@ -39,7 +39,7 @@ public class MasterKeyChangeRequest implements Message { @Order(2) byte[] digest; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public MasterKeyChangeRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java index 0e38787604a08..02edc0d90aa9f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheStatisticsModeChangeMessage.java @@ -21,11 +21,11 @@ import java.util.Collections; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -58,7 +58,7 @@ public class CacheStatisticsModeChangeMessage implements DiscoveryCustomMessage, byte flags; /** - * Constructor for {@link CoreMessagesProvider}. + * Constructor for {@link MessageFactory}. */ public CacheStatisticsModeChangeMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java index 720f57c44890a..e46e0cf6dcf5a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeFailureMessage.java @@ -25,7 +25,6 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; @@ -38,6 +37,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -68,7 +68,7 @@ public class ExchangeFailureMessage implements DiscoveryCustomMessage, Message { /** Actions to be done to rollback changes done before the exchange failure. */ private transient ExchangeActions exchangeRollbackActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ExchangeFailureMessage() { // No-op. } 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 7d47676273d35..acd4baab4f86d 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 @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.distributed; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.GridCacheMessage; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java index fc3f766a0c6fb..a12210c29cb77 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotOperationRequest.java @@ -22,11 +22,11 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -59,7 +59,7 @@ abstract class AbstractSnapshotOperationRequest implements Message { @Order(5) Set nodes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public AbstractSnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java index c16735839ff45..ffcf11b7a1a35 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/DataStreamerUpdatesHandlerResult.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class DataStreamerUpdatesHandlerResult implements Message { @@ -27,7 +27,7 @@ public class DataStreamerUpdatesHandlerResult implements Message { @Order(0) boolean streamerWarning; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public DataStreamerUpdatesHandlerResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java index 88215dce9104c..d614f2071d303 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerifyResult.java @@ -20,7 +20,6 @@ import java.util.Collection; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.pagemem.wal.record.DataEntry; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; @@ -30,6 +29,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @@ -58,7 +58,7 @@ public class IncrementalSnapshotVerifyResult implements MarshallableMessage { @Order(3) Collection exceptions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public IncrementalSnapshotVerifyResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java index 6ae8ff9aef638..41ab204b19355 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckHandlersResponse.java @@ -19,8 +19,8 @@ import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckHandlersResponse implements Message { @@ -28,7 +28,7 @@ public class SnapshotCheckHandlersResponse implements Message { @Order(0) Map>> perMetaResults; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckHandlersResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java index 504a437906928..f5489867ffcfe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckPartitionHashesResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotCheckPartitionHashesResponse implements MarshallableMessage @Order(0) byte[] perMetaResultsBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckPartitionHashesResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java index 5e4900a8fc65c..791a8f559694b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckProcessRequest.java @@ -20,10 +20,10 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class SnapshotCheckProcessRequest extends AbstractSnapshotOperationReques @Order(2) int incIdx; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckProcessRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java index fd6ea067aee39..693576027fd84 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCheckResponse.java @@ -21,10 +21,10 @@ import java.util.HashMap; import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** A DTO to transfer node's results for the both phases. */ @@ -37,7 +37,7 @@ public final class SnapshotCheckResponse implements Message { @Order(1) @Nullable Map errors; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotCheckResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java index d70ac1f46ac6b..02b5bd9b31f2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotHandlerResult.java @@ -18,9 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -38,7 +38,7 @@ public class SnapshotHandlerResult implements Message { @Order(1) ErrorMessage errMsg; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotHandlerResult() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java index 33b8f6dc3ec81..d45325ad093ad 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadataResponse.java @@ -20,10 +20,10 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotMetadataResponse implements MarshallableMessage { @@ -34,7 +34,7 @@ public class SnapshotMetadataResponse implements MarshallableMessage { /** */ private List metadata; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotMetadataResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java index ed3d0dacb8432..3b24db0c1a8e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationEndRequest.java @@ -20,12 +20,12 @@ import java.util.List; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -48,7 +48,7 @@ public class SnapshotOperationEndRequest implements Message { @Order(2) @Nullable List warnings; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationEndRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index bb93b19176809..25aa5fb10a57d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -21,9 +21,9 @@ import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class SnapshotOperationRequest extends AbstractSnapshotOperationRequest { @Order(7) boolean configOnly; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java index e9b1f434c70fd..5f023f2e3aedd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationResponse.java @@ -19,8 +19,8 @@ import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -29,7 +29,7 @@ public class SnapshotOperationResponse implements Message { @Order(0) Map> hndResults; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java index 0435d470e0a89..40ea2671f3946 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandlerResponse.java @@ -21,11 +21,11 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; import org.apache.ignite.internal.management.cache.PartitionKey; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecord; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMessage { @@ -36,7 +36,7 @@ public class SnapshotPartitionsVerifyHandlerResponse implements MarshallableMess @Order(0) byte[] resBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotPartitionsVerifyHandlerResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java index b9426cfb36f7e..2f658cf168073 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreOperationResponse.java @@ -22,11 +22,11 @@ import java.util.List; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Snapshot operation prepare response. */ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @@ -44,7 +44,7 @@ public class SnapshotRestoreOperationResponse implements MarshallableMessage { @Order(1) byte[] metasBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotRestoreOperationResponse() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java index 865f954b5f1ab..5f8b82713243c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreStartRequest.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** */ public class SnapshotRestoreStartRequest implements Message { @@ -29,7 +29,7 @@ public class SnapshotRestoreStartRequest implements Message { @Order(0) UUID reqId; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SnapshotRestoreStartRequest() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java index 23fce6364f11e..c7611c0b0bee8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotStartDiscoveryMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.distributed.InitMessage; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT; @@ -34,7 +34,7 @@ public class SnapshotStartDiscoveryMessage extends InitMessage allNodesMetrics; - /** Default constructor. Required for {@link CoreMessagesProvider}. */ + /** Default constructor. Required for {@link MessageFactory}. */ public ClusterMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java index 352ec9291cefc..b52a355b61083 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/NodeFullMetricsMessage.java @@ -21,10 +21,10 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Node compound metrics message. */ public class NodeFullMetricsMessage implements Message { @@ -36,7 +36,7 @@ public class NodeFullMetricsMessage implements Message { @Order(1) public Map cachesMetricsMsgs; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public NodeFullMetricsMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java index e6fd484913936..35bd6407a4b13 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasAckMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.metastorage.persistence; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; /** */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java index 117584b9fd0b9..f3817d43f175f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageCasMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -37,7 +37,7 @@ public class DistributedMetaStorageCasMessage extends DistributedMetaStorageUpda @Order(1) boolean matches; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public DistributedMetaStorageCasMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java index dd410581ac2f4..181f3f35751b4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateAckMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.metastorage.persistence; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java index 2127197afe69a..dabd0a791354b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metastorage/persistence/DistributedMetaStorageUpdateMessage.java @@ -19,12 +19,12 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -50,7 +50,7 @@ public class DistributedMetaStorageUpdateMessage implements DiscoveryCustomMessa @Order(3) byte[] valBytes; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public DistributedMetaStorageUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java index e9965f22b4d73..875432ccbfbbd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceChangeBatchRequest.java @@ -20,7 +20,6 @@ import java.util.Collection; import java.util.Collections; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; @@ -28,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class ServiceChangeBatchRequest implements DiscoveryCustomMessage, Messag @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceChangeBatchRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java index 5d6b775125fab..655f70dfd45ff 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResult.java @@ -21,11 +21,11 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -43,7 +43,7 @@ public class ServiceClusterDeploymentResult implements Message { @GridToStringInclude Map results; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceClusterDeploymentResult() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java index 20c4634047e91..f947d54131eec 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceClusterDeploymentResultBatch.java @@ -19,13 +19,13 @@ import java.util.Collection; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; @@ -55,7 +55,7 @@ public class ServiceClusterDeploymentResultBatch implements DiscoveryCustomMessa @GridToStringExclude @Nullable private ServiceDeploymentActions serviceDeploymentActions; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceClusterDeploymentResultBatch() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java index a243275bd04a5..4d1813477ad91 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceDeploymentRequest.java @@ -19,12 +19,12 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** @@ -38,7 +38,7 @@ public class ServiceDeploymentRequest extends ServiceChangeAbstractRequest imple @Order(0) byte[] cfgBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceDeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java index 99a85aef7cb9c..5a5911c65332f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/ServiceUndeploymentRequest.java @@ -17,16 +17,16 @@ package org.apache.ignite.internal.processors.service; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.NotNull; /** * Service undeployment request. */ public class ServiceUndeploymentRequest extends ServiceChangeAbstractRequest { - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public ServiceUndeploymentRequest() { } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java index 6ed2194d5a572..9639f4c843ec9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/FullMessage.java @@ -21,7 +21,6 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; @@ -29,6 +28,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -63,7 +63,7 @@ public class FullMessage implements DiscoveryCustomMessage, M @Order(4) Map err; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public FullMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java index b7aadc67bf089..cf9412cb587d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/InitMessage.java @@ -19,7 +19,6 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -28,6 +27,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -62,7 +62,7 @@ public class InitMessage implements Message, DiscoveryCustomM @Order(4) public boolean waitClnRes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public InitMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java index 38786258f74ba..ddd06005c3aeb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/SingleNodeMessage.java @@ -19,10 +19,10 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.ErrorMessage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -50,7 +50,7 @@ public class SingleNodeMessage implements Message { @Order(3) @Nullable ErrorMessage errMsg; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public SingleNodeMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java index e66fcb41f34a0..b4177fcfd7810 100644 --- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java +++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteProductVersion.java @@ -25,9 +25,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteVersionUtils; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; import org.jetbrains.annotations.NotNull; diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java index 5e36086bc2d9d..df25d13a17bd6 100644 --- a/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/plugin/extensions/communication/MessageFactoryProvider.java @@ -17,7 +17,7 @@ package org.apache.ignite.plugin.extensions.communication; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.Extension; /** diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java index 6735edf6602e4..b8eee2efbc59a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/GridNioServerWrapper.java @@ -48,12 +48,12 @@ import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteTooManyOpenFilesException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.managers.GridManager; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.tracing.GridTracingManager; import org.apache.ignite.internal.processors.metric.GridMetricManager; import org.apache.ignite.internal.processors.tracing.Tracing; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java index bf82966e14279..41583a1a3564c 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpConnectionRequestDiscoveryMessage.java @@ -19,12 +19,12 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -63,7 +63,7 @@ public TcpConnectionRequestDiscoveryMessage(UUID receiverNodeId, int connIdx) { this.connIdx = connIdx; } - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpConnectionRequestDiscoveryMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java index 4258926d5470b..7a609d2f7ebdf 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/internal/TcpHandshakeExecutor.java @@ -25,7 +25,7 @@ import javax.net.ssl.SSLException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.util.nio.ssl.BlockingSslHandler; import org.apache.ignite.internal.util.nio.ssl.GridSslMeta; import org.apache.ignite.internal.util.typedef.internal.U; diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java index 3d905dad3a536..8b62cf7adabf3 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetAddressMessage.java @@ -21,9 +21,9 @@ import java.net.UnknownHostException; import org.apache.ignite.IgniteException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Address utility container message. Is not a pure {@link TcpDiscoveryAbstractMessage}. */ public class InetAddressMessage implements Message { @@ -35,7 +35,7 @@ public class InetAddressMessage implements Message { @Order(1) byte[] addrBytes; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public InetAddressMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java index e792b9b21e8f5..f23e36f200d27 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/InetSocketAddressMessage.java @@ -19,8 +19,8 @@ import java.net.InetAddress; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Socket address utility container message. Is not a pure {@link TcpDiscoveryAbstractMessage}. */ public class InetSocketAddressMessage extends InetAddressMessage { @@ -29,7 +29,7 @@ public class InetSocketAddressMessage extends InetAddressMessage { int port; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public InetSocketAddressMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java index 01d937cde8028..2df35f55ce64a 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractTraceableMessage.java @@ -20,11 +20,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.tracing.messages.SpanContainer; import org.apache.ignite.internal.processors.tracing.messages.TraceableMessage; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -40,7 +40,7 @@ public abstract class TcpDiscoveryAbstractTraceableMessage extends TcpDiscoveryA @Nullable byte[] spanContainerBytes; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ protected TcpDiscoveryAbstractTraceableMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java index c7ee4aaececc4..bf3a97da951b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAuthFailedMessage.java @@ -20,9 +20,9 @@ import java.net.InetAddress; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Message telling joining node that its authentication failed. @@ -39,7 +39,7 @@ public class TcpDiscoveryAuthFailedMessage extends TcpDiscoveryAbstractMessage i @Order(1) UUID targetNodeId; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TcpDiscoveryAuthFailedMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java index 515e07397fe63..8851ae20bb478 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientMetricsUpdateMessage.java @@ -20,10 +20,10 @@ import java.util.UUID; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Metrics update message. @@ -38,7 +38,7 @@ public class TcpDiscoveryClientMetricsUpdateMessage extends TcpDiscoveryAbstract @Order(0) NodeMetricsMessage metricsMsg; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java index 140d3b9977b5b..7aa4afa0b0be8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientNodesMetricsMessage.java @@ -20,10 +20,10 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** Holds map of thick client or server metrics messages per node id. */ public class TcpDiscoveryClientNodesMetricsMessage implements Message { @@ -31,7 +31,7 @@ public class TcpDiscoveryClientNodesMetricsMessage implements Message { @Order(0) Map nodesMetricsMsgs; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientNodesMetricsMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java index ffda34cf60c00..6aa882964358d 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientReconnectMessage.java @@ -22,11 +22,11 @@ import java.util.Objects; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -49,7 +49,7 @@ public class TcpDiscoveryClientReconnectMessage extends TcpDiscoveryAbstractMess @Order(2) @Nullable TcpDiscoveryCollectionMessage pendingMsgsMsg; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryClientReconnectMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java index eea757b4a34a4..0d3c2b864aee6 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCollectionMessage.java @@ -24,7 +24,6 @@ import java.util.Map; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -32,6 +31,7 @@ import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -52,7 +52,7 @@ public class TcpDiscoveryCollectionMessage implements MarshallableMessage { @GridToStringExclude @Nullable byte[] marshallableMsgsBytes; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryCollectionMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java index dbea16218c2eb..2f938710cd091 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryConnectionCheckMessage.java @@ -17,9 +17,9 @@ package org.apache.ignite.spi.discovery.tcp.messages; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; /** @@ -32,7 +32,7 @@ public class TcpDiscoveryConnectionCheckMessage extends TcpDiscoveryAbstractMess private static final long serialVersionUID = 0L; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryConnectionCheckMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java index b134410faee5c..1e2b713998c75 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java @@ -21,13 +21,11 @@ import java.util.UUID; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; -import org.apache.ignite.internal.managers.CoreMessagesProvider; -import org.apache.ignite.internal.managers.discovery.IncompleteDeserializationException; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; import org.jetbrains.annotations.Nullable; @@ -54,7 +52,7 @@ public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractTraceabl volatile @Nullable Message serMsg; /** - * Constructor for {@link CoreMessagesProvider}. + * Constructor for {@link MessageFactory}. */ public TcpDiscoveryCustomEventMessage() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java index 9e503b87594af..d553cf5910976 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeRequest.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -40,7 +40,7 @@ public class TcpDiscoveryHandshakeRequest extends TcpDiscoveryAbstractMessage im @Nullable String dcId; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryHandshakeRequest() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java index 21f207e6fc6b3..e3381ab5aaeb4 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java @@ -21,10 +21,10 @@ import java.util.Collection; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -47,7 +47,7 @@ public class TcpDiscoveryHandshakeResponse extends TcpDiscoveryAbstractMessage i @Nullable Collection redirectAddrsMsgs; /** - * Default constructor for {@link CoreMessagesProvider}. + * Default constructor for {@link MessageFactory}. */ public TcpDiscoveryHandshakeResponse() { // No-op. diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java index 1f4213a92191b..522f66e5dfbe8 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryMetricsUpdateMessage.java @@ -25,7 +25,6 @@ import org.apache.ignite.cache.CacheMetrics; import org.apache.ignite.cluster.ClusterMetrics; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.cluster.CacheMetricsMessage; import org.apache.ignite.internal.processors.cluster.NodeFullMetricsMessage; import org.apache.ignite.internal.processors.cluster.NodeMetricsMessage; @@ -33,6 +32,7 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -68,7 +68,7 @@ public class TcpDiscoveryMetricsUpdateMessage extends TcpDiscoveryAbstractMessag @Order(2) @Nullable Set clientNodeIds; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryMetricsUpdateMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java index 4b4b148af9091..e47dac2443c13 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java @@ -21,10 +21,10 @@ import java.util.Map; import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket; import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode; import org.jetbrains.annotations.Nullable; @@ -69,7 +69,7 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractTraceableM @Order(5) long gridStartTime; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryNodeAddedMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java index 9e50d472ddfce..9f6071d629af0 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeLeftMessage.java @@ -18,8 +18,8 @@ package org.apache.ignite.spi.discovery.tcp.messages; import java.util.UUID; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * Sent by node that is stopping to coordinator across the ring, @@ -31,7 +31,7 @@ public class TcpDiscoveryNodeLeftMessage extends TcpDiscoveryAbstractTraceableMe /** */ private static final long serialVersionUID = 0L; - /** Constructor for {@link CoreMessagesProvider}. */ + /** Constructor for {@link MessageFactory}. */ public TcpDiscoveryNodeLeftMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java index 6067262b4d0eb..6e07a1167a072 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryRingLatencyCheckMessage.java @@ -19,9 +19,9 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; /** * @@ -38,7 +38,7 @@ public class TcpDiscoveryRingLatencyCheckMessage extends TcpDiscoveryAbstractMes @Order(1) int curHops; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public TcpDiscoveryRingLatencyCheckMessage() { // No-op. } diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java index 14fe94552ab4b..c9d8385b2560f 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryStatusCheckMessage.java @@ -23,9 +23,9 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** @@ -58,7 +58,7 @@ public class TcpDiscoveryStatusCheckMessage extends TcpDiscoveryAbstractMessage @Order(2) int status; - /** Empty constructor for {@link CoreMessagesProvider}. */ + /** Empty constructor for {@link MessageFactory}. */ public TcpDiscoveryStatusCheckMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java index 30c44be358a39..a4c9164388911 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.Message; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java index 5f1a56622ede0..32ef435f32077 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/TestNestedContainersMessage.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -40,7 +39,7 @@ class TestNestedContainersMessage implements Message { @Order(2) Map nestedArr; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestNestedContainersMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java index b146af2c53967..93e846ce703b1 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java @@ -22,11 +22,11 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; import org.apache.ignite.internal.direct.state.DirectMessageState; import org.apache.ignite.internal.direct.stream.DirectByteBufferStream; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GroupPartitionIdPair; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java index e5be7dd50c8d2..8c9a0e2b8e19d 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java @@ -19,7 +19,7 @@ import java.util.UUID; import org.apache.commons.lang3.reflect.FieldUtils; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.processors.cache.distributed.dht.PartitionUpdateCountersMessage; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java index 53a4cad334664..c34eb96cf4d02 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteMessageFactoryImplTest.java @@ -20,7 +20,7 @@ import java.nio.ByteBuffer; import org.apache.ignite.IgniteException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java index 4197222ab2caf..a9df4dbc68678 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/MessageDirectTypeIdConflictTest.java @@ -21,7 +21,7 @@ import java.util.concurrent.Callable; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.plugin.AbstractTestPluginProvider; import org.apache.ignite.plugin.ExtensionRegistry; import org.apache.ignite.plugin.PluginContext; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java index 4dc2ddedd0b34..c3f51d523a05a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java @@ -17,7 +17,7 @@ package org.apache.ignite.internal.managers.discovery; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.AbstractMessageSerializationTest; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java index 2473dacf942e5..15ddd54040c5a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java @@ -30,9 +30,9 @@ import org.apache.ignite.cache.query.QueryCursor; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheObjectImpl; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java index b960d0f886dbe..da1b7ca67fc8f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestIntegerMessage.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.util.distributed; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -27,7 +26,7 @@ public class TestIntegerMessage implements Message { @Order(0) int val; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestIntegerMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java index 4dc4ebee4bd14..57940c1e17ffe 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/util/distributed/TestUuidMessage.java @@ -19,7 +19,6 @@ import java.util.UUID; import org.apache.ignite.internal.Order; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.plugin.extensions.communication.Message; /** */ @@ -28,7 +27,7 @@ public class TestUuidMessage implements Message { @Order(0) UUID val; - /** Default constructor for {@link CoreMessagesProvider}. */ + /** Default constructor for {@link MessageFactory}. */ public TestUuidMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index 10615b39022a4..f82c844bda973 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 31a4f880344b8..1bc2358cc7e2f 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -35,9 +35,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteNodeAttributes; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java index 4f448c0f3c1b5..a604da0edb7f0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java @@ -25,9 +25,9 @@ import org.apache.ignite.IgniteLogger; import org.apache.ignite.IgniteSystemProperties; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.metric.MetricRegistryImpl; import org.apache.ignite.internal.util.typedef.F; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 62fcd6d55569c..6a5d26eb26619 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -36,9 +36,9 @@ import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.nio.GridCommunicationClient; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index 648d7c3febb5c..de46f4b674a88 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -27,7 +27,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index 9e1c6145459b7..633bd332eaf28 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -30,9 +30,9 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java index 70fa1b6c967b3..d5add388704b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSkipWaitHandshakeOnClientTest.java @@ -20,8 +20,8 @@ import java.io.InputStream; import java.net.Socket; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.testframework.GridTestUtils; import org.apache.ignite.testframework.ListeningTestLogger; import org.apache.ignite.testframework.LogListener; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index 71aaae35c5f38..06d657a8a3bea 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -28,7 +28,7 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.managers.CoreMessagesProvider; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor; import org.apache.ignite.internal.util.GridConcurrentHashSet; diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java index 8eeae66fac4e5..a26013a586ff4 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/DummyCustomDiscoveryMessage.java @@ -19,9 +19,9 @@ import org.apache.ignite.internal.Order; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.jetbrains.annotations.Nullable; /** */ @@ -30,7 +30,7 @@ public class DummyCustomDiscoveryMessage implements DiscoveryCustomMessage, Mess @Order(0) IgniteUuid id = IgniteUuid.randomUuid(); - /** Constructor for {@link DiscoveryMessageFactory}. */ + /** Constructor for {@link MessageFactory}. */ public DummyCustomDiscoveryMessage() { // No-op. } diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java index 859c0979beb61..decf2bf4b5c3e 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java @@ -20,8 +20,8 @@ import java.io.IOException; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; -import org.apache.ignite.internal.managers.discovery.DiscoveryMessageFactory; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpiInternalListener; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.extensions.communication.MessageFactory; @@ -121,7 +121,7 @@ public void messageFactory(MessageFactoryProvider msgFactoryProvider) { assert !started(); this.msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new DiscoveryMessageFactory(jdk(), U.resolveClassLoader(ignite().configuration())), + new CoreMessagesProvider(jdk(), U.resolveClassLoader(ignite().configuration())), msgFactoryProvider }); } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index 62391811edc5e..ce5ac8e79f69f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -37,10 +37,10 @@ import org.apache.ignite.events.Event; import org.apache.ignite.events.TaskEvent; import org.apache.ignite.internal.ClusterMetricsSnapshot; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.GridTopic; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.GridIoPolicy; import org.apache.ignite.internal.managers.communication.GridIoUserMessage; import org.apache.ignite.internal.managers.communication.GridMessageListener; diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java index 77b75d6922694..fec34ef4a336d 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java @@ -26,9 +26,9 @@ import java.nio.ByteBuffer; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.CoreMessagesProvider; import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.marshaller.Marshaller; From 9da474f32efcf1990972c76e7288010ed05946a7 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 15:09:58 +0300 Subject: [PATCH 4/9] minor --- .../ignite/internal/CoreMessagesProvider.java | 518 +++++++++--------- 1 file changed, 258 insertions(+), 260 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 82285e93e50cb..34e3d74b48de7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -320,322 +320,320 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { msgIdx = 5000; // We don't use the code‑generated serializer for CompressedMessage - serialization is highly customized. factory.register(msgIdx++, CompressedMessage::new); - register(factory, ErrorMessage.class); - register(factory, InetSocketAddressMessage.class); - register(factory, InetAddressMessage.class); - register(factory, TcpDiscoveryNode.class); - register(factory, IgniteProductVersion.class); - register(factory, DiscoveryDataPacket.class); - register(factory, GridByteArrayList.class); - register(factory, CacheVersionedValue.class); - register(factory, GridCacheVersion.class); - register(factory, GridCacheVersionEx.class); + register(ErrorMessage.class); + register(InetSocketAddressMessage.class); + register(InetAddressMessage.class); + register(TcpDiscoveryNode.class); + register(IgniteProductVersion.class); + register(DiscoveryDataPacket.class); + register(GridByteArrayList.class); + register(CacheVersionedValue.class); + register(GridCacheVersion.class); + register(GridCacheVersionEx.class); msgIdx = 5500; - register(factory, TcpDiscoveryCollectionMessage.class); + register(TcpDiscoveryCollectionMessage.class); // [5700 - 5900]: Discovery originated messages. msgIdx = 5700; - register(factory, TcpDiscoveryHandshakeRequest.class); - register(factory, TcpDiscoveryHandshakeResponse.class); - register(factory, TcpDiscoveryJoinRequestMessage.class); - register(factory, TcpDiscoveryNodeAddedMessage.class); - register(factory, TcpDiscoveryNodeAddFinishedMessage.class); - register(factory, TcpDiscoveryNodeLeftMessage.class); - register(factory, TcpDiscoveryNodeFailedMessage.class); - register(factory, TcpDiscoveryConnectionCheckMessage.class); - register(factory, TcpDiscoveryPingRequest.class); - register(factory, TcpDiscoveryPingResponse.class); - register(factory, TcpDiscoveryClientPingRequest.class); - register(factory, TcpDiscoveryClientPingResponse.class); - register(factory, TcpDiscoveryClientAckResponse.class); - register(factory, TcpDiscoveryClientReconnectMessage.class); - register(factory, TcpDiscoveryDiscardMessage.class); - register(factory, TcpDiscoveryCheckFailedMessage.class); - register(factory, TcpDiscoveryLoopbackProblemMessage.class); - register(factory, TcpDiscoveryRingLatencyCheckMessage.class); - register(factory, TcpDiscoveryDuplicateIdMessage.class); - register(factory, TcpDiscoveryCustomEventMessage.class); - register(factory, TcpDiscoveryServerOnlyCustomEventMessage.class); + register(TcpDiscoveryHandshakeRequest.class); + register(TcpDiscoveryHandshakeResponse.class); + register(TcpDiscoveryJoinRequestMessage.class); + register(TcpDiscoveryNodeAddedMessage.class); + register(TcpDiscoveryNodeAddFinishedMessage.class); + register(TcpDiscoveryNodeLeftMessage.class); + register(TcpDiscoveryNodeFailedMessage.class); + register(TcpDiscoveryConnectionCheckMessage.class); + register(TcpDiscoveryPingRequest.class); + register(TcpDiscoveryPingResponse.class); + register(TcpDiscoveryClientPingRequest.class); + register(TcpDiscoveryClientPingResponse.class); + register(TcpDiscoveryClientAckResponse.class); + register(TcpDiscoveryClientReconnectMessage.class); + register(TcpDiscoveryDiscardMessage.class); + register(TcpDiscoveryCheckFailedMessage.class); + register(TcpDiscoveryLoopbackProblemMessage.class); + register(TcpDiscoveryRingLatencyCheckMessage.class); + register(TcpDiscoveryDuplicateIdMessage.class); + register(TcpDiscoveryCustomEventMessage.class); + register(TcpDiscoveryServerOnlyCustomEventMessage.class); msgIdx = 5900; - register(factory, TcpDiscoveryStatusCheckMessage.class); + register(TcpDiscoveryStatusCheckMessage.class); // [6000 - 6200]: Snapshot operation messages. Most of them originally come from Discovery. msgIdx = 6000; - register(factory, SnapshotStartDiscoveryMessage.class); - register(factory, SnapshotCheckProcessRequest.class); - register(factory, SnapshotOperationRequest.class); - register(factory, SnapshotOperationEndRequest.class); - register(factory, SnapshotRestoreStartRequest.class); - register(factory, SnapshotOperationResponse.class); - register(factory, SnapshotHandlerResult.class); - register(factory, SnapshotCheckResponse.class); - register(factory, SnapshotPartitionsVerifyHandlerResponse.class); - register(factory, SnapshotRestoreOperationResponse.class); - register(factory, SnapshotMetadataResponse.class); - register(factory, SnapshotCheckPartitionHashesResponse.class); - register(factory, SnapshotCheckHandlersResponse.class); - register(factory, SnapshotFilesRequestMessage.class); - register(factory, SnapshotFilesFailureMessage.class); - register(factory, IncrementalSnapshotVerifyResult.class); - register(factory, IncrementalSnapshotAwareMessage.class); + register(SnapshotStartDiscoveryMessage.class); + register(SnapshotCheckProcessRequest.class); + register(SnapshotOperationRequest.class); + register(SnapshotOperationEndRequest.class); + register(SnapshotRestoreStartRequest.class); + register(SnapshotOperationResponse.class); + register(SnapshotHandlerResult.class); + register(SnapshotCheckResponse.class); + register(SnapshotPartitionsVerifyHandlerResponse.class); + register(SnapshotRestoreOperationResponse.class); + register(SnapshotMetadataResponse.class); + register(SnapshotCheckPartitionHashesResponse.class); + register(SnapshotCheckHandlersResponse.class); + register(SnapshotFilesRequestMessage.class); + register(SnapshotFilesFailureMessage.class); + register(IncrementalSnapshotVerifyResult.class); + register(IncrementalSnapshotAwareMessage.class); // [6300 - 6400]: Services messages. Most of them originally come from Discovery. msgIdx = 6300; - register(factory, ServiceDeploymentProcessId.class); - register(factory, ServiceSingleNodeDeploymentResult.class); - register(factory, ServiceClusterDeploymentResult.class); - register(factory, ServiceDeploymentRequest.class); - register(factory, ServiceUndeploymentRequest.class); - register(factory, ServiceClusterDeploymentResultBatch.class); - register(factory, ServiceChangeBatchRequest.class); - register(factory, ServiceSingleNodeDeploymentResultBatch.class); + register(ServiceDeploymentProcessId.class); + register(ServiceSingleNodeDeploymentResult.class); + register(ServiceClusterDeploymentResult.class); + register(ServiceDeploymentRequest.class); + register(ServiceUndeploymentRequest.class); + register(ServiceClusterDeploymentResultBatch.class); + register(ServiceChangeBatchRequest.class); + register(ServiceSingleNodeDeploymentResultBatch.class); // [6500 - 6700]: DiscoveryCustomMessage msgIdx = 6500; - register(factory, TcpConnectionRequestDiscoveryMessage.class); - register(factory, DistributedMetaStorageUpdateMessage.class); - register(factory, DistributedMetaStorageUpdateAckMessage.class); - register(factory, DistributedMetaStorageCasMessage.class); - register(factory, DistributedMetaStorageCasAckMessage.class); - register(factory, FullMessage.class); - register(factory, InitMessage.class); - register(factory, CacheStatisticsModeChangeMessage.class); - register(factory, SecurityAwareCustomMessageWrapper.class); - register(factory, MetadataRemoveAcceptedMessage.class); - register(factory, MetadataRemoveProposedMessage.class); - register(factory, WalStateFinishMessage.class); - register(factory, WalStateProposeMessage.class); - register(factory, MetadataUpdateAcceptedMessage.class); - register(factory, TxTimeoutOnPartitionMapExchangeChangeMessage.class); - register(factory, UserAcceptedMessage.class); - register(factory, UserProposedMessage.class); - register(factory, ChangeGlobalStateFinishMessage.class); - register(factory, StopRoutineAckDiscoveryMessage.class); - register(factory, StopRoutineDiscoveryMessage.class); - register(factory, CacheAffinityChangeMessage.class); - register(factory, ClientCacheChangeDiscoveryMessage.class); - register(factory, MappingAcceptedMessage.class); - register(factory, MappingProposedMessage.class); - register(factory, ExchangeFailureMessage.class); - register(factory, CacheStatisticsClearMessage.class); - register(factory, ClientCacheChangeDummyDiscoveryMessage.class); - register(factory, DynamicCacheChangeBatch.class); + register(TcpConnectionRequestDiscoveryMessage.class); + register(DistributedMetaStorageUpdateMessage.class); + register(DistributedMetaStorageUpdateAckMessage.class); + register(DistributedMetaStorageCasMessage.class); + register(DistributedMetaStorageCasAckMessage.class); + register(FullMessage.class); + register(InitMessage.class); + register(CacheStatisticsModeChangeMessage.class); + register(SecurityAwareCustomMessageWrapper.class); + register(MetadataRemoveAcceptedMessage.class); + register(MetadataRemoveProposedMessage.class); + register(WalStateFinishMessage.class); + register(WalStateProposeMessage.class); + register(MetadataUpdateAcceptedMessage.class); + register(TxTimeoutOnPartitionMapExchangeChangeMessage.class); + register(UserAcceptedMessage.class); + register(UserProposedMessage.class); + register(ChangeGlobalStateFinishMessage.class); + register(StopRoutineAckDiscoveryMessage.class); + register(StopRoutineDiscoveryMessage.class); + register(CacheAffinityChangeMessage.class); + register(ClientCacheChangeDiscoveryMessage.class); + register(MappingAcceptedMessage.class); + register(MappingProposedMessage.class); + register(ExchangeFailureMessage.class); + register(CacheStatisticsClearMessage.class); + register(ClientCacheChangeDummyDiscoveryMessage.class); + register(DynamicCacheChangeBatch.class); // [10000 - 10200]: Transaction and lock related messages. Most of the originally comes from Communication. msgIdx = 10000; - register(factory, TxInfo.class); - register(factory, TxEntriesInfo.class); - register(factory, TxLock.class); - register(factory, TxLocksRequest.class); - register(factory, TxLocksResponse.class); - register(factory, IgniteTxKey.class); - register(factory, IgniteTxEntry.class); - register(factory, TxEntryValueHolder.class); - register(factory, GridCacheTxRecoveryRequest.class); - register(factory, GridCacheTxRecoveryResponse.class); - register(factory, GridDistributedTxFinishRequest.class); - register(factory, GridDistributedTxFinishResponse.class); - register(factory, GridDistributedTxPrepareRequest.class); - register(factory, GridDistributedTxPrepareResponse.class); - register(factory, GridDhtTxFinishRequest.class); - register(factory, GridDhtTxFinishResponse.class); - register(factory, GridDhtTxPrepareRequest.class); - register(factory, GridDhtTxPrepareResponse.class); - register(factory, GridNearTxFinishRequest.class); - register(factory, GridNearTxFinishResponse.class); - register(factory, GridNearTxPrepareRequest.class); - register(factory, GridNearTxPrepareResponse.class); - register(factory, GridDhtLockRequest.class); - register(factory, GridDhtLockResponse.class); - register(factory, GridDhtUnlockRequest.class); - register(factory, GridNearLockRequest.class); - register(factory, GridNearLockResponse.class); - register(factory, GridNearUnlockRequest.class); - register(factory, GridDistributedLockRequest.class); - register(factory, GridDistributedLockResponse.class); - register(factory, GridDhtTxOnePhaseCommitAckRequest.class); - register(factory, TransactionAttributesAwareRequest.class); + register(TxInfo.class); + register(TxEntriesInfo.class); + register(TxLock.class); + register(TxLocksRequest.class); + register(TxLocksResponse.class); + register(IgniteTxKey.class); + register(IgniteTxEntry.class); + register(TxEntryValueHolder.class); + register(GridCacheTxRecoveryRequest.class); + register(GridCacheTxRecoveryResponse.class); + register(GridDistributedTxFinishRequest.class); + register(GridDistributedTxFinishResponse.class); + register(GridDistributedTxPrepareRequest.class); + register(GridDistributedTxPrepareResponse.class); + register(GridDhtTxFinishRequest.class); + register(GridDhtTxFinishResponse.class); + register(GridDhtTxPrepareRequest.class); + register(GridDhtTxPrepareResponse.class); + register(GridNearTxFinishRequest.class); + register(GridNearTxFinishResponse.class); + register(GridNearTxPrepareRequest.class); + register(GridNearTxPrepareResponse.class); + register(GridDhtLockRequest.class); + register(GridDhtLockResponse.class); + register(GridDhtUnlockRequest.class); + register(GridNearLockRequest.class); + register(GridNearLockResponse.class); + register(GridNearUnlockRequest.class); + register(GridDistributedLockRequest.class); + register(GridDistributedLockResponse.class); + register(GridDhtTxOnePhaseCommitAckRequest.class); + register(TransactionAttributesAwareRequest.class); // [10300 - 10500]: Cache, DHT messages. msgIdx = 10300; - register(factory, GridDhtForceKeysRequest.class); - register(factory, GridDhtForceKeysResponse.class); - register(factory, GridDhtAtomicDeferredUpdateResponse.class); - register(factory, GridDhtAtomicUpdateRequest.class); - register(factory, GridDhtAtomicUpdateResponse.class); - register(factory, GridNearAtomicFullUpdateRequest.class); - register(factory, GridDhtAtomicSingleUpdateRequest.class); - register(factory, GridNearAtomicUpdateResponse.class); - register(factory, GridNearAtomicSingleUpdateRequest.class); - register(factory, GridNearAtomicSingleUpdateInvokeRequest.class); - register(factory, GridNearAtomicSingleUpdateFilterRequest.class); - register(factory, GridNearAtomicCheckUpdateRequest.class); - register(factory, NearCacheUpdates.class); - register(factory, GridNearGetRequest.class); - register(factory, GridNearGetResponse.class); - register(factory, GridNearSingleGetRequest.class); - register(factory, GridNearSingleGetResponse.class); - register(factory, GridDhtAtomicNearResponse.class); - register(factory, GridCacheTtlUpdateRequest.class); - register(factory, GridCacheReturn.class); - register(factory, GridCacheEntryInfo.class); - register(factory, CacheInvokeDirectResult.class); - register(factory, GridCacheRawVersionedEntry.class); - register(factory, CacheEvictionEntry.class); - register(factory, CacheEntryPredicateAdapter.class); - register(factory, GridContinuousMessage.class); - register(factory, ContinuousRoutineStartResultMessage.class); - register(factory, UpdateErrors.class); - register(factory, LatchAckMessage.class); - register(factory, AtomicApplicationAttributesAwareRequest.class); - register(factory, StartRequestData.class); - register(factory, StartRoutineDiscoveryMessage.class); - register(factory, StartRoutineAckDiscoveryMessage.class); - register(factory, StartRoutineDiscoveryMessageV2.class); + register(GridDhtForceKeysRequest.class); + register(GridDhtForceKeysResponse.class); + register(GridDhtAtomicDeferredUpdateResponse.class); + register(GridDhtAtomicUpdateRequest.class); + register(GridDhtAtomicUpdateResponse.class); + register(GridNearAtomicFullUpdateRequest.class); + register(GridDhtAtomicSingleUpdateRequest.class); + register(GridNearAtomicUpdateResponse.class); + register(GridNearAtomicSingleUpdateRequest.class); + register(GridNearAtomicSingleUpdateInvokeRequest.class); + register(GridNearAtomicSingleUpdateFilterRequest.class); + register(GridNearAtomicCheckUpdateRequest.class); + register(NearCacheUpdates.class); + register(GridNearGetRequest.class); + register(GridNearGetResponse.class); + register(GridNearSingleGetRequest.class); + register(GridNearSingleGetResponse.class); + register(GridDhtAtomicNearResponse.class); + register(GridCacheTtlUpdateRequest.class); + register(GridCacheReturn.class); + register(GridCacheEntryInfo.class); + register(CacheInvokeDirectResult.class); + register(GridCacheRawVersionedEntry.class); + register(CacheEvictionEntry.class); + register(CacheEntryPredicateAdapter.class); + register(GridContinuousMessage.class); + register(ContinuousRoutineStartResultMessage.class); + register(UpdateErrors.class); + register(LatchAckMessage.class); + register(AtomicApplicationAttributesAwareRequest.class); + register(StartRequestData.class); + register(StartRoutineDiscoveryMessage.class); + register(StartRoutineAckDiscoveryMessage.class); + register(StartRoutineDiscoveryMessageV2.class); // [10600-10800]: Affinity & partition maps. msgIdx = 10600; - register(factory, GridDhtAffinityAssignmentRequest.class); - register(factory, GridDhtAffinityAssignmentResponse.class); - register(factory, CacheGroupAffinityMessage.class); - register(factory, ExchangeInfo.class); - register(factory, PartitionUpdateCountersMessage.class); - register(factory, CachePartitionPartialCountersMap.class); - register(factory, IgniteDhtDemandedPartitionsMap.class); - register(factory, CachePartitionFullCountersMap.class); - register(factory, GroupPartitionIdPair.class); - register(factory, IgniteDhtPartitionHistorySuppliersMap.class); - register(factory, GridPartitionStateMap.class); - register(factory, GridDhtPartitionMap.class); - register(factory, GridDhtPartitionFullMap.class); - register(factory, GridDhtPartitionExchangeId.class); - register(factory, GridCheckpointRequest.class); - register(factory, GridDhtPartitionDemandMessage.class); - register(factory, GridDhtPartitionSupplyMessage.class); - register(factory, GridDhtPartitionsFullMessage.class); - register(factory, GridDhtPartitionsSingleMessage.class); - register(factory, GridDhtPartitionsSingleRequest.class); + register(GridDhtAffinityAssignmentRequest.class); + register(GridDhtAffinityAssignmentResponse.class); + register(CacheGroupAffinityMessage.class); + register(ExchangeInfo.class); + register(PartitionUpdateCountersMessage.class); + register(CachePartitionPartialCountersMap.class); + register(IgniteDhtDemandedPartitionsMap.class); + register(CachePartitionFullCountersMap.class); + register(GroupPartitionIdPair.class); + register(IgniteDhtPartitionHistorySuppliersMap.class); + register(GridPartitionStateMap.class); + register(GridDhtPartitionMap.class); + register(GridDhtPartitionFullMap.class); + register(GridDhtPartitionExchangeId.class); + register(GridCheckpointRequest.class); + register(GridDhtPartitionDemandMessage.class); + register(GridDhtPartitionSupplyMessage.class); + register(GridDhtPartitionsFullMessage.class); + register(GridDhtPartitionsSingleMessage.class); + register(GridDhtPartitionsSingleRequest.class); // [10900-11100]: Query, schema and SQL related messages. msgIdx = 10900; - register(factory, SchemaAlterTableAddColumnOperation.class); - register(factory, SchemaIndexCreateOperation.class); - register(factory, SchemaIndexDropOperation.class); - register(factory, SchemaAlterTableDropColumnOperation.class); - register(factory, SchemaAddQueryEntityOperation.class); - register(factory, SchemaOperationStatusMessage.class); - register(factory, SchemaProposeDiscoveryMessage.class); - register(factory, SchemaFinishDiscoveryMessage.class); - register(factory, QueryField.class); - register(factory, GridCacheSqlQuery.class); - register(factory, GridCacheQueryRequest.class); - register(factory, GridCacheQueryResponse.class); - register(factory, GridQueryCancelRequest.class); - register(factory, GridQueryFailResponse.class); - register(factory, GridQueryNextPageRequest.class); - register(factory, GridQueryNextPageResponse.class); - register(factory, GridQueryKillRequest.class); - register(factory, GridQueryKillResponse.class); - register(factory, IndexKeyDefinition.class); - register(factory, IndexKeyTypeSettings.class); - register(factory, IndexQueryResultMeta.class); - register(factory, StatisticsKeyMessage.class); - register(factory, StatisticsDecimalMessage.class); - register(factory, StatisticsObjectData.class); - register(factory, StatisticsColumnData.class); - register(factory, StatisticsRequest.class); - register(factory, StatisticsResponse.class); - register(factory, CacheContinuousQueryBatchAck.class); - register(factory, CacheContinuousQueryEntry.class); + register(SchemaAlterTableAddColumnOperation.class); + register(SchemaIndexCreateOperation.class); + register(SchemaIndexDropOperation.class); + register(SchemaAlterTableDropColumnOperation.class); + register(SchemaAddQueryEntityOperation.class); + register(SchemaOperationStatusMessage.class); + register(SchemaProposeDiscoveryMessage.class); + register(SchemaFinishDiscoveryMessage.class); + register(QueryField.class); + register(GridCacheSqlQuery.class); + register(GridCacheQueryRequest.class); + register(GridCacheQueryResponse.class); + register(GridQueryCancelRequest.class); + register(GridQueryFailResponse.class); + register(GridQueryNextPageRequest.class); + register(GridQueryNextPageResponse.class); + register(GridQueryKillRequest.class); + register(GridQueryKillResponse.class); + register(IndexKeyDefinition.class); + register(IndexKeyTypeSettings.class); + register(IndexQueryResultMeta.class); + register(StatisticsKeyMessage.class); + register(StatisticsDecimalMessage.class); + register(StatisticsObjectData.class); + register(StatisticsColumnData.class); + register(StatisticsRequest.class); + register(StatisticsResponse.class); + register(CacheContinuousQueryBatchAck.class); + register(CacheContinuousQueryEntry.class); // [11200 - 11300]: Compute, distributed process messages. msgIdx = 11200; - register(factory, GridJobCancelRequest.class); - register(factory, GridJobExecuteRequest.class); - register(factory, GridJobExecuteResponse.class); - register(factory, GridJobSiblingsRequest.class); - register(factory, GridJobSiblingsResponse.class); - register(factory, GridTaskCancelRequest.class); - register(factory, GridTaskSessionRequest.class); - register(factory, GridTaskResultRequest.class); - register(factory, GridTaskResultResponse.class); - register(factory, JobStealingRequest.class); - register(factory, SingleNodeMessage.class); + register(GridJobCancelRequest.class); + register(GridJobExecuteRequest.class); + register(GridJobExecuteResponse.class); + register(GridJobSiblingsRequest.class); + register(GridJobSiblingsResponse.class); + register(GridTaskCancelRequest.class); + register(GridTaskSessionRequest.class); + register(GridTaskResultRequest.class); + register(GridTaskResultResponse.class); + register(JobStealingRequest.class); + register(SingleNodeMessage.class); // [11500 - 11600]: IO, networking messages. msgIdx = NODE_ID_MSG_TYPE; - register(factory, NodeIdMessage.class); - register(factory, HandshakeMessage.class); - register(factory, HandshakeWaitMessage.class); - register(factory, GridIoMessage.class); + register(NodeIdMessage.class); + register(HandshakeMessage.class); + register(HandshakeWaitMessage.class); + register(GridIoMessage.class); factory.register(msgIdx++, IgniteIoTestMessage::new); - register(factory, GridIoUserMessage.class); - register(factory, GridIoSecurityAwareMessage.class); - register(factory, RecoveryLastReceivedMessage.class); - register(factory, TcpInverseConnectionResponseMessage.class); - register(factory, SessionChannelMessage.class); + register(GridIoUserMessage.class); + register(GridIoSecurityAwareMessage.class); + register(RecoveryLastReceivedMessage.class); + register(TcpInverseConnectionResponseMessage.class); + register(SessionChannelMessage.class); // [11700 - 11800]: Datastreamer messages. msgIdx = 11700; - register(factory, DataStreamerUpdatesHandlerResult.class); - register(factory, DataStreamerEntry.class); - register(factory, DataStreamerRequest.class); - register(factory, DataStreamerResponse.class); + register(DataStreamerUpdatesHandlerResult.class); + register(DataStreamerEntry.class); + register(DataStreamerRequest.class); + register(DataStreamerResponse.class); // [11900 - 12000]: Metrics, monitoring messages. msgIdx = 11900; - register(factory, CacheMetricsMessage.class); - register(factory, NodeMetricsMessage.class); - register(factory, NodeFullMetricsMessage.class); - register(factory, ClusterMetricsUpdateMessage.class); - register(factory, TcpDiscoveryClientNodesMetricsMessage.class); - register(factory, TcpDiscoveryMetricsUpdateMessage.class); - register(factory, TcpDiscoveryClientMetricsUpdateMessage.class); + register(CacheMetricsMessage.class); + register(NodeMetricsMessage.class); + register(NodeFullMetricsMessage.class); + register(ClusterMetricsUpdateMessage.class); + register(TcpDiscoveryClientNodesMetricsMessage.class); + register(TcpDiscoveryMetricsUpdateMessage.class); + register(TcpDiscoveryClientMetricsUpdateMessage.class); // [12000 - 12100]: Authentication, security messages. msgIdx = 12000; - register(factory, User.class); - register(factory, UserManagementOperation.class); - register(factory, UserManagementOperationFinishedMessage.class); - register(factory, UserAuthenticateRequestMessage.class); - register(factory, UserAuthenticateResponseMessage.class); - register(factory, TcpDiscoveryAuthFailedMessage.class); + register(User.class); + register(UserManagementOperation.class); + register(UserManagementOperationFinishedMessage.class); + register(UserAuthenticateRequestMessage.class); + register(UserAuthenticateResponseMessage.class); + register(TcpDiscoveryAuthFailedMessage.class); // [12200 - 12300]: Binary, classloading and marshalling messages. msgIdx = 12200; - register(factory, GridDeploymentInfoBean.class); - register(factory, GridDeploymentRequest.class); - register(factory, GridDeploymentResponse.class); - register(factory, MissingMappingRequestMessage.class); - register(factory, MissingMappingResponseMessage.class); - register(factory, MetadataRequestMessage.class); - register(factory, MetadataResponseMessage.class); - register(factory, MarshallerMappingItem.class); - register(factory, BinaryMetadataVersionInfo.class); + register(GridDeploymentInfoBean.class); + register(GridDeploymentRequest.class); + register(GridDeploymentResponse.class); + register(MissingMappingRequestMessage.class); + register(MissingMappingResponseMessage.class); + register(MetadataRequestMessage.class); + register(MetadataResponseMessage.class); + register(MarshallerMappingItem.class); + register(BinaryMetadataVersionInfo.class); // [12400 - 12500]: Encryption messages. msgIdx = 12400; - register(factory, GenerateEncryptionKeyRequest.class); - register(factory, GenerateEncryptionKeyResponse.class); - register(factory, ChangeCacheEncryptionRequest.class); - register(factory, MasterKeyChangeRequest.class); + register(GenerateEncryptionKeyRequest.class); + register(GenerateEncryptionKeyResponse.class); + register(ChangeCacheEncryptionRequest.class); + register(MasterKeyChangeRequest.class); // [13000 - 13300]: Control, diagnostincs and other messages. msgIdx = 13000; - register(factory, GridEventStorageMessage.class); - register(factory, ChangeGlobalStateMessage.class); - register(factory, GridChangeGlobalStateMessageResponse.class); - register(factory, IgniteDiagnosticRequest.class); - register(factory, IgniteDiagnosticResponse.class); - register(factory, WalStateAckMessage.class); - - this.factory = null; + register(GridEventStorageMessage.class); + register(ChangeGlobalStateMessage.class); + register(GridChangeGlobalStateMessageResponse.class); + register(IgniteDiagnosticRequest.class); + register(IgniteDiagnosticResponse.class); + register(WalStateAckMessage.class); } /** Registers message incrementing {@link #msgIdx}. */ - private void register(MessageFactory factory, Class cls) { + private void register(Class cls) { Constructor ctor; MessageSerializer serializer; From 6f4709d71f2724fb2375d57d5cf5a095cde2865c Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 15:11:47 +0300 Subject: [PATCH 5/9] minor --- .../java/org/apache/ignite/internal/CoreMessagesProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 34e3d74b48de7..48e89ebc089ca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -642,7 +642,7 @@ private void register(Class cls) { boolean marshallable = MarshallableMessage.class.isAssignableFrom(cls); - Class serCls = Class.forName(cls.getName() + (marshallable ? "Marshallable" : "") + "Serializer"); + Class serCls = Class.forName(cls.getName() + (marshallable ? "MarshallableSerializer" : "Serializer")); serializer = marshallable ? (MessageSerializer)serCls.getConstructor(Marshaller.class, ClassLoader.class).newInstance(marsh, clsLdr) From 9a3c6af077058e1566f879c70c60d462dcc624f8 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 15:20:14 +0300 Subject: [PATCH 6/9] typo --- .../java/org/apache/ignite/internal/CoreMessagesProvider.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index 48e89ebc089ca..bba654595efcf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -649,7 +649,7 @@ private void register(Class cls) { : (MessageSerializer)serCls.getConstructor().newInstance(); } catch (Exception e) { - throw new IgniteException("Failted to register message of type " + cls.getSimpleName(), e); + throw new IgniteException("Failed to register message of type " + cls.getSimpleName(), e); } factory.register( @@ -659,7 +659,7 @@ private void register(Class cls) { return ctor.newInstance(); } catch (Exception e) { - throw new IgniteException("Failted to create message of type " + cls.getSimpleName(), e); + throw new IgniteException("Failed to create message of type " + cls.getSimpleName(), e); } }, serializer From e51c2c1b1d63ca2bbd548fcf6031a4745c889239 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Wed, 8 Apr 2026 17:39:30 +0300 Subject: [PATCH 7/9] marshaller fix --- .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java index 724472761bdf5..9e067f84439f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java +++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java @@ -595,7 +595,7 @@ public void setClientReconnectDisabled(boolean clientReconnectDisabled) { setLocalAddress(ignite.configuration().getLocalHost()); setAddressResolver(ignite.configuration().getAddressResolver()); - marsh = ((IgniteEx)ignite).context().marshaller(); + marsh = ((IgniteEx)ignite).context().marshallerContext().jdkMarshaller(); } } From c9390c50a430efd6dba4446571f2d91363563a91 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 9 Apr 2026 15:07:54 +0300 Subject: [PATCH 8/9] reimpl --- .../ignite/internal/CoreMessagesProvider.java | 33 +++++++++++++------ .../apache/ignite/internal/IgniteKernal.java | 3 +- .../direct/DirectMarshallingMessagesTest.java | 2 +- .../communication/CompressedMessageTest.java | 2 +- ...CommunicationMessageSerializationTest.java | 2 +- ...niteDiscoveryMessageSerializationTest.java | 2 +- ...acheContinuousQueryImmutableEntryTest.java | 2 +- .../GridAbstractCommunicationSelfTest.java | 2 +- ...unicationSpiConcurrentConnectSelfTest.java | 2 +- ...GridTcpCommunicationSpiConfigSelfTest.java | 2 +- ...CommunicationSpiMultithreadedSelfTest.java | 2 +- ...cpCommunicationSpiRecoveryAckSelfTest.java | 2 +- ...idTcpCommunicationSpiRecoverySelfTest.java | 2 +- ...mmunicationRecoveryAckClosureSelfTest.java | 2 +- .../discovery/tcp/TestTcpDiscoverySpi.java | 2 +- .../testframework/GridSpiTestContext.java | 3 +- .../discovery/zk/ZookeeperDiscoverySpi.java | 4 ++- .../zk/internal/DiscoveryMessageParser.java | 18 ++++------ .../zk/internal/ZookeeperDiscoveryImpl.java | 11 ++++--- 19 files changed, 57 insertions(+), 41 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index bba654595efcf..c7e4a7618ab39 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -19,6 +19,7 @@ import java.lang.reflect.Constructor; import org.apache.ignite.IgniteException; +import org.apache.ignite.internal.binary.BinaryMarshaller; import org.apache.ignite.internal.cache.query.index.IndexQueryResultMeta; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyDefinition; import org.apache.ignite.internal.cache.query.index.sorted.IndexKeyTypeSettings; @@ -227,8 +228,10 @@ import org.apache.ignite.internal.util.distributed.FullMessage; import org.apache.ignite.internal.util.distributed.InitMessage; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteProductVersion; import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.plugin.extensions.communication.MarshallableMessage; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; @@ -285,11 +288,14 @@ public class CoreMessagesProvider implements MessageFactoryProvider { /** Handshake wait message type. */ public static final short HANDSHAKE_WAIT_MSG_TYPE = HANDSHAKE_MSG_TYPE + 1; - /** Custom data marshaller. */ - private final Marshaller marsh; + /** Binary marshaller. */ + private final Marshaller schemaAwareMarhaller; - /** Class loader for the custom data marshalling. */ - private final ClassLoader clsLdr; + /** Binary marshaller. */ + private final Marshaller schemaLessMarshaller; + + /** Resolved classloader. */ + private final ClassLoader resolvedClsLdr; /** */ private short msgIdx; @@ -298,12 +304,14 @@ public class CoreMessagesProvider implements MessageFactoryProvider { private @Nullable MessageFactory factory; /** - * @param marsh Custom data marshaller. - * @param clsLdr Class loader for the custom data marshalling. + * @param schemaAwareMarhaller Schema-aware marshaller like {@link BinaryMarshaller}. + * @param schemaLessMarshaller Pure, schemaless marshaller like {@link JdkMarshaller}. + * @param resolvedClsLdr Resolved classloader. */ - public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { - this.marsh = marsh; - this.clsLdr = clsLdr; + public CoreMessagesProvider(Marshaller schemaAwareMarhaller, Marshaller schemaLessMarshaller, ClassLoader resolvedClsLdr) { + this.schemaAwareMarhaller = schemaAwareMarhaller; + this.schemaLessMarshaller = schemaLessMarshaller; + this.resolvedClsLdr = resolvedClsLdr; } /** {@inheritDoc} */ @@ -632,8 +640,13 @@ public CoreMessagesProvider(Marshaller marsh, ClassLoader clsLdr) { register(WalStateAckMessage.class); } - /** Registers message incrementing {@link #msgIdx}. */ + /** */ private void register(Class cls) { + register(cls, schemaLessMarshaller, U.gridClassLoader()); + } + + /** Registers message using ane incrementing {@link #msgIdx} as the message id/type. */ + private void register(Class cls, Marshaller marsh, ClassLoader clsLdr) { Constructor ctor; MessageSerializer serializer; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 2dd8268f0dd10..84f76d21449f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1320,7 +1320,8 @@ private void initMessageFactory() throws IgniteCheckedException { List compMsgs = new ArrayList<>(); - compMsgs.add(new CoreMessagesProvider(ctx.marshaller(), U.resolveClassLoader(ctx.config()))); + compMsgs.add(new CoreMessagesProvider(ctx.marshaller(), ctx.marshallerContext().jdkMarshaller(), + U.resolveClassLoader(ctx.config()))); for (IgniteComponentType compType : IgniteComponentType.values()) { MessageFactoryProvider f = compType.messageFactory(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java index a4c9164388911..ea37b6abe4545 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/direct/DirectMarshallingMessagesTest.java @@ -46,7 +46,7 @@ public class DirectMarshallingMessagesTest extends GridCommonAbstractTest { /** Message factory. */ private final MessageFactory msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new CoreMessagesProvider(jdk(), U.gridClassLoader()), + new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), factory -> factory.register( TestNestedContainersMessage.TYPE, TestNestedContainersMessage::new, diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java index 93e846ce703b1..05b699a31ef2a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/CompressedMessageTest.java @@ -47,7 +47,7 @@ public class CompressedMessageTest { @Test public void testWriteReadHugeMessage() { MessageFactory msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{ - new CoreMessagesProvider(jdk(), U.gridClassLoader())}); + new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader())}); DirectMessageWriter writer = new DirectMessageWriter(msgFactory); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java index 8c9a0e2b8e19d..4bb4364236fa6 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoCommunicationMessageSerializationTest.java @@ -34,7 +34,7 @@ public class IgniteIoCommunicationMessageSerializationTest extends AbstractMessageSerializationTest { /** {@inheritDoc} */ @Override protected MessageFactoryProvider messageFactory() { - return new CoreMessagesProvider(jdk(), U.gridClassLoader()); + return new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java index c3f51d523a05a..f79bf80d8950e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/IgniteDiscoveryMessageSerializationTest.java @@ -28,6 +28,6 @@ public class IgniteDiscoveryMessageSerializationTest extends AbstractMessageSerializationTest { /** {@inheritDoc} */ @Override protected MessageFactoryProvider messageFactory() { - return new CoreMessagesProvider(jdk(), U.gridClassLoader()); + return new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()); } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java index 15ddd54040c5a..a41490c0e405f 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/IgniteCacheContinuousQueryImmutableEntryTest.java @@ -150,7 +150,7 @@ public void testCacheContinuousQueryEntrySerialization() { e0.markFiltered(); IgniteMessageFactoryImpl msgFactory = - new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), U.gridClassLoader())}); + new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader())}); ByteBuffer buf = ByteBuffer.allocate(4096); DirectMessageWriter writer = new DirectMessageWriter(msgFactory); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java index f82c844bda973..1ca7217b67dca 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java @@ -154,7 +154,7 @@ private void startSpis() throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new CoreMessagesProvider(jdk(), U.gridClassLoader()), customMessageFactory()})); + new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), customMessageFactory()})); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java index 1bc2358cc7e2f..997d5b59fd4e6 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java @@ -434,7 +434,7 @@ private void startSpis(MessageListener lsnr) throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java index a604da0edb7f0..3f2d90924bf28 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java @@ -252,7 +252,7 @@ private TcpCommunicationSpi initializeSpi(GridSpiTestContext ctx, node.setId(rsrcs.getNodeId()); ctx.messageFactory(new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{ - new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY})); + new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY})); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java index 6a5d26eb26619..c043c068fc434 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java @@ -468,7 +468,7 @@ private int getSpiCount() { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.timeoutProcessor(timeoutProcessor); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java index de46f4b674a88..56c6daceca5d2 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java @@ -400,7 +400,7 @@ private void startSpis(int ackCnt, int idleTimeout, int queueLimit) throws Excep GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java index 633bd332eaf28..d4f00e425a428 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java @@ -729,7 +729,7 @@ private void startSpis() throws Exception { GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java index 06d657a8a3bea..58a321e21c8a9 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java @@ -453,7 +453,7 @@ private void startSpis(int ackCnt, int idleTimeout, int queueLimit) throws Excep GridSpiTestContext ctx = initSpiContext(); ctx.messageFactory(new IgniteMessageFactoryImpl( - new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) + new MessageFactoryProvider[] {new CoreMessagesProvider(jdk(), jdk(), U.gridClassLoader()), GRID_TEST_MESSAGE_FACTORY}) ); ctx.setLocalNode(node); diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java index decf2bf4b5c3e..2aaabee1aeb71 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestTcpDiscoverySpi.java @@ -121,7 +121,7 @@ public void messageFactory(MessageFactoryProvider msgFactoryProvider) { assert !started(); this.msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { - new CoreMessagesProvider(jdk(), U.resolveClassLoader(ignite().configuration())), + new CoreMessagesProvider(jdk(), jdk(), U.resolveClassLoader(ignite().configuration())), msgFactoryProvider }); } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java index ce5ac8e79f69f..4eb5d79e0b6f9 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java @@ -554,7 +554,8 @@ public void triggerEvent(Event evt) { /** {@inheritDoc} */ @Override public MessageFactory messageFactory() { if (factory == null) - factory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), U.gridClassLoader())}); + factory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[]{new CoreMessagesProvider(jdk(), jdk(), + U.gridClassLoader())}); return factory; } diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java index 54347b18970dd..e10799e1b42f7 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/ZookeeperDiscoverySpi.java @@ -460,7 +460,9 @@ public DiscoverySpiNodeAuthenticator getAuthenticator() { lsnr, exchange, stats, - ((IgniteEx)ignite).context().marshallerContext().jdkMarshaller()); + ((IgniteEx)ignite).context().marshallerContext().jdkMarshaller(), + ((IgniteEx)ignite).context().messageFactory() + ); registerMBean(igniteInstanceName, new ZookeeperDiscoverySpiMBeanImpl(this), ZookeeperDiscoverySpiMBean.class); diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java index fec34ef4a336d..a3944ac917c84 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/DiscoveryMessageParser.java @@ -26,16 +26,12 @@ import java.nio.ByteBuffer; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; -import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.direct.DirectMessageReader; import org.apache.ignite.internal.direct.DirectMessageWriter; -import org.apache.ignite.internal.managers.communication.IgniteMessageFactoryImpl; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.plugin.extensions.communication.MessageFactory; -import org.apache.ignite.plugin.extensions.communication.MessageFactoryProvider; import org.apache.ignite.plugin.extensions.communication.MessageSerializer; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage; @@ -57,15 +53,15 @@ public class DiscoveryMessageParser { private static final int MSG_BUFFER_SIZE = 100; /** */ - private final MessageFactory msgFactory; + private final JdkMarshaller jdkMarshaller; /** */ - private final Marshaller marsh; + private final MessageFactory msgFactory; /** */ - public DiscoveryMessageParser(Marshaller marsh) { - this.marsh = marsh; - msgFactory = new IgniteMessageFactoryImpl(new MessageFactoryProvider[] { new CoreMessagesProvider(marsh, U.gridClassLoader()) }); + public DiscoveryMessageParser(JdkMarshaller jdkMarshaller, MessageFactory msgFactory) { + this.jdkMarshaller = jdkMarshaller; + this.msgFactory = msgFactory; } /** Marshals discovery message to bytes array. */ @@ -81,7 +77,7 @@ public byte[] marshalZip(DiscoverySpiCustomMessage msg) { else { out.write(JAVA_SERIALIZATION); - U.marshal(marsh, msg, out); + U.marshal(jdkMarshaller, msg, out); } } catch (Exception e) { @@ -100,7 +96,7 @@ public DiscoverySpiCustomMessage unmarshalZip(byte[] bytes) { byte mode = (byte)in.read(); if (mode == JAVA_SERIALIZATION) - return U.unmarshal(marsh, in, U.gridClassLoader()); + return U.unmarshal(jdkMarshaller, in, U.gridClassLoader()); if (MESSAGE_SERIALIZATION != mode) throw new IOException("Received unexpected byte while reading discovery message: " + mode); diff --git a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java index 2cac8fd62f26b..e056f1d96a82a 100644 --- a/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java +++ b/modules/zookeeper/src/main/java/org/apache/ignite/spi/discovery/zk/internal/ZookeeperDiscoveryImpl.java @@ -77,6 +77,7 @@ import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.plugin.extensions.communication.MessageFactory; import org.apache.ignite.plugin.security.SecurityCredentials; import org.apache.ignite.spi.IgniteNodeValidationResult; import org.apache.ignite.spi.IgniteSpiException; @@ -228,7 +229,8 @@ public class ZookeeperDiscoveryImpl { * @param lsnr Discovery events listener. * @param exchange Discovery data exchange. * @param stats Zookeeper DiscoverySpi statistics collector. - * @param marsh Marshaller. + * @param jdkMarshaller JDK Marshaller. + * @param msgFactory Message Factory. */ public ZookeeperDiscoveryImpl( ZookeeperDiscoverySpi spi, @@ -239,7 +241,8 @@ public ZookeeperDiscoveryImpl( DiscoverySpiListener lsnr, DiscoverySpiDataExchange exchange, ZookeeperDiscoveryStatistics stats, - JdkMarshaller marsh + JdkMarshaller jdkMarshaller, + MessageFactory msgFactory ) { assert locNode.id() != null && locNode.isLocal() : locNode; @@ -254,7 +257,7 @@ public ZookeeperDiscoveryImpl( this.lsnr = lsnr; this.exchange = exchange; this.clientReconnectEnabled = locNode.isClient() && !spi.isClientReconnectDisabled(); - this.marsh = marsh; + this.marsh = jdkMarshaller; int evtsAckThreshold = IgniteSystemProperties.getInteger(IGNITE_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD, DFLT_ZOOKEEPER_DISCOVERY_SPI_ACK_THRESHOLD); @@ -266,7 +269,7 @@ public ZookeeperDiscoveryImpl( this.stats = stats; - msgParser = new DiscoveryMessageParser(marsh); + msgParser = new DiscoveryMessageParser(jdkMarshaller, msgFactory); } /** From f147a484431a32f3dc78990ef90f5afe392e3a01 Mon Sep 17 00:00:00 2001 From: Vladimir Steshin Date: Thu, 9 Apr 2026 22:10:17 +0300 Subject: [PATCH 9/9] test fix --- .../org/apache/ignite/internal/CoreMessagesProvider.java | 5 +++++ .../java/org/apache/ignite/spi/MessagesPluginProvider.java | 3 ++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java index c7e4a7618ab39..d7c9f4aec9f26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/CoreMessagesProvider.java @@ -288,6 +288,9 @@ public class CoreMessagesProvider implements MessageFactoryProvider { /** Handshake wait message type. */ public static final short HANDSHAKE_WAIT_MSG_TYPE = HANDSHAKE_MSG_TYPE + 1; + /** */ + public static final short MAX_MESSAGE_ID = 15_000; + /** Binary marshaller. */ private final Marshaller schemaAwareMarhaller; @@ -638,6 +641,8 @@ public CoreMessagesProvider(Marshaller schemaAwareMarhaller, Marshaller schemaLe register(IgniteDiagnosticRequest.class); register(IgniteDiagnosticResponse.class); register(WalStateAckMessage.class); + + assert msgIdx <= MAX_MESSAGE_ID; } /** */ diff --git a/modules/core/src/test/java/org/apache/ignite/spi/MessagesPluginProvider.java b/modules/core/src/test/java/org/apache/ignite/spi/MessagesPluginProvider.java index 5134383656e71..bda1aebcd2285 100644 --- a/modules/core/src/test/java/org/apache/ignite/spi/MessagesPluginProvider.java +++ b/modules/core/src/test/java/org/apache/ignite/spi/MessagesPluginProvider.java @@ -19,6 +19,7 @@ import java.util.function.Supplier; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.CoreMessagesProvider; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.plugin.AbstractTestPluginProvider; import org.apache.ignite.plugin.ExtensionRegistry; @@ -39,7 +40,7 @@ public class MessagesPluginProvider extends AbstractTestPluginProvider { @SafeVarargs public MessagesPluginProvider(Class... msgs) { msgFactoryProvider = f -> { - short directType = 10_000; + short directType = CoreMessagesProvider.MAX_MESSAGE_ID + 1; for (Class msg : msgs) { Supplier msgSupp = () -> {