PakhomovAlexander commented on code in PR #1780:
URL: https://github.com/apache/ignite-3/pull/1780#discussion_r1138625519
##########
modules/code-deployment/src/main/java/org/apache/ignite/internal/deployunit/DeploymentManagerImpl.java:
##########
@@ -226,7 +227,8 @@ public CompletableFuture<Void> undeployAsync(String id,
Version version) {
}).thenApply(logicalTopologySnapshot -> {
for (ClusterNode node : logicalTopologySnapshot.nodes()) {
clusterService.messagingService()
- .invoke(node, UndeployUnitRequestImpl.builder()
+ .invoke(node, DEPLOYMENT_UNITS,
Review Comment:
I think `DEPLOYMENT_UNITS_CHANNEL` is a better naming here. Reading the code
now it is not clear that u send the message through the special channel.
##########
modules/network-api/src/main/java/org/apache/ignite/network/ChannelType.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.network;
+
+/**
+ * Channel type with unique identifier.
+ */
+public enum ChannelType {
+ DEFAULT((short) 0),
+ DEPLOYMENT_UNITS((short) 1),
Review Comment:
add empty line, please.
##########
modules/network/src/integrationTest/java/org/apache/ignite/internal/network/netty/ItConnectionManagerTest.java:
##########
@@ -296,6 +300,61 @@ public void testStopTwice() throws Exception {
server.close();
}
+ @Test
+ public void testOneConnectionType() throws Exception {
+ int size = 10000000;
+ char[] chars = new char[size];
+ Arrays.fill(chars, 'a');
+ String bigText = new String(chars);
+
+
+ ConnectionManagerWrapper server1 = startManager(4000);
+ ConnectionManagerWrapper server2 = startManager(4001);
+
+ NettySender sender = server1.openChannelTo(server2,
ChannelType.DEFAULT).get(3, TimeUnit.SECONDS);
+
+
+ TestMessage bigMessage =
messageFactory.testMessage().msg(bigText).build();
+ TestMessage msg = messageFactory.testMessage().msg("test").build();
+
+ CompletableFuture<Void> send1 = sender.send(new
OutNetworkObject(bigMessage, Collections.emptyList()));
+ CompletableFuture<Void> send2 = sender.send(new OutNetworkObject(msg,
Collections.emptyList()));
+
+ assertThat(send2, willCompleteSuccessfully());
+ assertThat(send1.isDone(), is(true));
+
+ server1.close();
+ server2.close();
+ }
+
+ @Test
+ public void testTwoConnectionTypes() throws Exception {
+ int size = 1000000000;
+ char[] chars = new char[size];
+ Arrays.fill(chars, 'a');
+ String bigText = new String(chars);
+
+
+ ConnectionManagerWrapper server1 = startManager(4000);
+ ConnectionManagerWrapper server2 = startManager(4001);
+
+ NettySender sender1 = server1.openChannelTo(server2,
ChannelType.DEFAULT).get(3, TimeUnit.SECONDS);
+ NettySender sender2 = server1.openChannelTo(server2,
ChannelType.TEST).get(3, TimeUnit.SECONDS);
+
+
+ TestMessage bigMessage =
messageFactory.testMessage().msg(bigText).build();
+ TestMessage msg = messageFactory.testMessage().msg("test").build();
+
+ CompletableFuture<Void> send1 = sender1.send(new
OutNetworkObject(bigMessage, Collections.emptyList()));
+ CompletableFuture<Void> send2 = sender2.send(new OutNetworkObject(msg,
Collections.emptyList()));
+
+ assertThat(send2, willCompleteSuccessfully());
+ assertThat(send1.isDone(), is(false));
+
+ server1.close();
+ server2.close();
Review Comment:
Could you close the servers in AfterEach block? In case the assert is wrong
they will hang.
##########
modules/network/src/main/java/org/apache/ignite/internal/network/netty/ConnectorKey.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.network.netty;
+
+import org.apache.ignite.network.ChannelType;
+
+/**
+ * Composite key object for connectors.
+ *
+ * @param <T> identifier type.
+ */
+public class ConnectorKey<T> {
+ private final T id;
+ private final ChannelType type;
Review Comment:
empty line
##########
modules/network/src/integrationTest/java/org/apache/ignite/internal/network/netty/ItConnectionManagerTest.java:
##########
@@ -296,6 +300,61 @@ public void testStopTwice() throws Exception {
server.close();
}
+ @Test
+ public void testOneConnectionType() throws Exception {
+ int size = 10000000;
+ char[] chars = new char[size];
+ Arrays.fill(chars, 'a');
+ String bigText = new String(chars);
+
+
+ ConnectionManagerWrapper server1 = startManager(4000);
+ ConnectionManagerWrapper server2 = startManager(4001);
+
+ NettySender sender = server1.openChannelTo(server2,
ChannelType.DEFAULT).get(3, TimeUnit.SECONDS);
+
+
+ TestMessage bigMessage =
messageFactory.testMessage().msg(bigText).build();
+ TestMessage msg = messageFactory.testMessage().msg("test").build();
+
+ CompletableFuture<Void> send1 = sender.send(new
OutNetworkObject(bigMessage, Collections.emptyList()));
+ CompletableFuture<Void> send2 = sender.send(new OutNetworkObject(msg,
Collections.emptyList()));
+
+ assertThat(send2, willCompleteSuccessfully());
+ assertThat(send1.isDone(), is(true));
Review Comment:
I understood what you are testing here but it's not obvious. Could you
please change the naming like: `firstSend`, `secondSend`. Because order matters
here.
Also, `bigMessage` is ok but why there is no `smallMassage`?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]