You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2021/11/19 09:10:41 UTC

[ignite-3] branch ignite-15307 updated: Add NettyBootstrapFactory

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

ptupitsyn pushed a commit to branch ignite-15307
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/ignite-15307 by this push:
     new 591c4c7  Add NettyBootstrapFactory
591c4c7 is described below

commit 591c4c7cdee828067f19b90de4fb7c5267ca467e
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Fri Nov 19 12:10:33 2021 +0300

    Add NettyBootstrapFactory
---
 modules/client-handler/pom.xml                     |  5 ++
 .../ignite/client/handler/ClientHandlerModule.java | 27 +++++--
 .../ignite/network/NettyBootstrapFactory.java      | 86 ++++++++++++++++++++++
 .../org/apache/ignite/internal/app/IgniteImpl.java |  5 +-
 4 files changed, 116 insertions(+), 7 deletions(-)

diff --git a/modules/client-handler/pom.xml b/modules/client-handler/pom.xml
index 0ecb6aa..2c1fa68 100644
--- a/modules/client-handler/pom.xml
+++ b/modules/client-handler/pom.xml
@@ -58,6 +58,11 @@
             <artifactId>ignite-calcite</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-network</artifactId>
+        </dependency>
+
         <!-- 3rd party dependencies -->
         <dependency>
             <groupId>com.google.code.gson</groupId>
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
index 3cb6cfc..6708ec5 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/ClientHandlerModule.java
@@ -32,9 +32,11 @@ import org.apache.ignite.configuration.schemas.clientconnector.ClientConnectorCo
 import org.apache.ignite.internal.client.proto.ClientMessageDecoder;
 import org.apache.ignite.internal.configuration.ConfigurationRegistry;
 import org.apache.ignite.internal.manager.IgniteComponent;
+import org.apache.ignite.internal.network.netty.ConnectionManager;
 import org.apache.ignite.internal.processors.query.calcite.QueryProcessor;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.lang.IgniteLogger;
+import org.apache.ignite.network.NettyBootstrapFactory;
 import org.apache.ignite.table.manager.IgniteTables;
 import org.jetbrains.annotations.Nullable;
 
@@ -56,22 +58,35 @@ public class ClientHandlerModule implements IgniteComponent {
 
     /** Processor. */
     private QueryProcessor processor;
-
+    
+    /** Connection manager. */
+    private ConnectionManager connectionManager;
+    
+    /** Netty bootstrap factory. */
+    private NettyBootstrapFactory bootstrapFactory;
+    
     /**
      * Constructor.
      *
-     * @param processor    Sql query processor.
-     * @param igniteTables Ignite.
-     * @param registry     Configuration registry.
+     * @param processor         Sql query processor.
+     * @param igniteTables      Ignite.
+     * @param registry          Configuration registry.
+     * @param bootstrapFactory  Bootstrap factory.
      */
-    public ClientHandlerModule(QueryProcessor processor, IgniteTables igniteTables, ConfigurationRegistry registry) {
+    public ClientHandlerModule(
+            QueryProcessor processor,
+            IgniteTables igniteTables,
+            ConfigurationRegistry registry,
+            NettyBootstrapFactory bootstrapFactory) {
         assert igniteTables != null;
         assert registry != null;
         assert processor != null;
-
+        assert bootstrapFactory != null;
+        
         this.processor = processor;
         this.igniteTables = igniteTables;
         this.registry = registry;
+        this.bootstrapFactory = bootstrapFactory;
     }
 
     /** {@inheritDoc} */
diff --git a/modules/network/src/main/java/org/apache/ignite/network/NettyBootstrapFactory.java b/modules/network/src/main/java/org/apache/ignite/network/NettyBootstrapFactory.java
new file mode 100644
index 0000000..85f8b84
--- /dev/null
+++ b/modules/network/src/main/java/org/apache/ignite/network/NettyBootstrapFactory.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+import io.netty.bootstrap.Bootstrap;
+import io.netty.channel.ChannelOption;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import org.apache.ignite.configuration.schemas.network.NetworkView;
+import org.apache.ignite.configuration.schemas.network.OutboundView;
+import org.apache.ignite.internal.network.netty.NamedNioEventLoopGroup;
+
+/**
+ * Netty bootstrap factory.
+ */
+public class NettyBootstrapFactory {
+    /** Node consistent id. */
+    private String consistentId;
+    
+    /** Network configuration. */
+    private NetworkView networkConfiguration;
+    
+    /** Server boss socket channel handler event loop group. */
+    private final EventLoopGroup bossGroup;
+    
+    /** Server work socket channel handler event loop group. */
+    private final EventLoopGroup workerGroup;
+    
+    /** Client socket channel handler event loop group. */
+    private final EventLoopGroup clientWorkerGroup;
+    
+    /**
+     * Constructor.
+     *
+     * @param networkConfiguration Network configuration.
+     * @param consistentId         Consistent id of this node.
+     */
+    public NettyBootstrapFactory(
+            NetworkView networkConfiguration,
+            String consistentId
+    ) {
+        assert consistentId != null;
+        assert networkConfiguration != null;
+        
+        this.consistentId = consistentId;
+        this.networkConfiguration = networkConfiguration;
+        
+        this.bossGroup = NamedNioEventLoopGroup.create(consistentId + "-srv-accept");
+        this.workerGroup = NamedNioEventLoopGroup.create(consistentId + "-srv-worker");
+        this.clientWorkerGroup = NamedNioEventLoopGroup.create(consistentId + "-client");
+    }
+    
+    /**
+     * Creates bootstrap for outbound client connections.
+     *
+     * @return Bootstrap.
+     */
+    public Bootstrap createClientBootstrap() {
+        OutboundView clientConfiguration = networkConfiguration.outbound();
+        Bootstrap clientBootstrap = new Bootstrap();
+        
+        clientBootstrap.group(clientWorkerGroup)
+                .channel(NioSocketChannel.class)
+                // See NettyServer#start for netty configuration details.
+                .option(ChannelOption.SO_KEEPALIVE, clientConfiguration.soKeepAlive())
+                .option(ChannelOption.SO_LINGER, clientConfiguration.soLinger())
+                .option(ChannelOption.TCP_NODELAY, clientConfiguration.tcpNoDelay());
+        
+        return clientBootstrap;
+    }
+}
diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 0bd11d3..7d3f6cd 100644
--- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -218,7 +218,10 @@ public class IgniteImpl implements Ignite {
 
         restModule = new RestModule(nodeCfgMgr, clusterCfgMgr);
 
-        clientHandlerModule = new ClientHandlerModule(qryEngine, distributedTblMgr, nodeCfgMgr.configurationRegistry());
+        // TODO: Create a common class to hold EventLoopGroup, and create all bootstraps.
+        // - Put it into network-api
+        // - Inject where needed
+        clientHandlerModule = new ClientHandlerModule(qryEngine, distributedTblMgr, nodeCfgMgr.configurationRegistry(), null);
     }
 
     /**