8000 Enabling configuring IP Address for Dotnet Backend by Niharikadutta · Pull Request #537 · dotnet/spark · GitHub
[go: up one dir, main page]

Skip to content

Enabling configuring IP Address for Dotnet Backend #537

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 18 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Adding environment variable for dotnet backend ip address
  • Loading branch information
Niharikadutta committed Jun 14, 2020
commit 8b3fd40b970eb480fb5d6677e29518242fdb9e40
3 changes: 2 additions & 1 deletion src/csharp/Microsoft.Spark/Interop/Ipc/JvmBridge.cs
Original file line number Diff line number Diff line change
Expand Up @@ -51,8 +51,9 @@ private ISocketWrapper GetConnection()
{
if (!_sockets.TryDequeue(out ISocketWrapper socket))
{
IPEndPoint dotnetBackendIPEndpoint = SparkEnvironment.ConfigurationService.GetBackendIPEndpoint();
socket = SocketFactory.CreateSocket();
socket.Connect(IPAddress.Loopback, _portNumber);
socket.Connect(dotnetBackendIPEndpoint.Address, dotnetBackendIPEndpoint.Port);
}

return socket;
Expand Down
10 changes: 1 addition & 9 deletions src/csharp/Microsoft.Spark/Network/DefaultSocketWrapper.cs
Original file line number Diff line number Diff line change
Expand Up @@ -30,15 +30,7 @@ internal sealed class DefaultSocketWrapper : ISocketWrapper
public DefaultSocketWrapper() :
this(new Socket(AddressFamily.InterNetwork, SocketType.Stream, ProtocolType.Tcp))
{
string dotnetBackendIPAddress = SparkEnvironment.ConfigurationService.GetBackendIPAddress();
if (dotnetBackendIPAddress == "localhost" || dotnetBackendIPAddress == null)
{
_innerSocket.Bind(new IPEndPoint(IPAddress.Loopback, 0));
}
else
{
_innerSocket.Bind(new IPEndPoint(IPAddress.Parse(dotnetBackendIPAddress), 0));
}
_innerSocket.Bind(new IPEndPoint(IPAddress.Any, 0));
}

/// <summary>
Expand Down
4 changes: 3 additions & 1 deletion src/csharp/Microsoft.Spark/RDD.cs
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@
using System.Collections.Generic;
using System.Linq;
using System.Net;
using Microsoft.Spark.Interop;
using Microsoft.Spark.Interop.Ipc;
using Microsoft.Spark.Network;
using Microsoft.Spark.Utils;
Expand Down Expand Up @@ -262,7 +263,8 @@ public IEnumerable<T> Collect()
{
(int port, string secret) = CollectAndServe();
using ISocketWrapper socket = SocketFactory.CreateSocket();
socket.Connect(IPAddress.Loopback, port, secret);
IPEndPoint dotnetBackendIPEndpoint = SparkEnvironment.ConfigurationService.GetBackendIPEndpoint();
socket.Connect(dotnetBackendIPEndpoint.Address, port, secret);

var collector = new RDD.Collector();
System.IO.Stream stream = socket.InputStream;
Expand Down
26 changes: 25 additions & 1 deletion src/csharp/Microsoft.Spark/Services/ConfigurationService.cs
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@

using System;
using System.IO;
using System.Net;
using System.Runtime.InteropServices;

namespace Microsoft.Spark.Services
Expand All @@ -20,7 +21,7 @@ internal sealed class ConfigurationService : IConfigurationService
"spark.dotnet.worker.writeBufferSize";

private const string DotnetBackendPortEnvVarName = "DOTNETBACKEND_PORT";
private const string DotnetBackendIPAddressEnvVarName = "DOTNETBACKEND_IP_ADDRESS";
private const string DotnetBackendIPAddressEnvVarName = "DOTNET_SPARK_BACKEND_IP_ADDRESS";
private const int DotnetBackendDebugPort = 5567;

private static readonly string s_procBaseFileName = "Microsoft.Spark.Worker";
Expand All @@ -34,6 +35,29 @@ internal sealed class ConfigurationService : IConfigurationService

private string _workerPath;

/// <summary>
/// Returns the IP Endpoint for socket communication between JVM and CLR.
/// </summary>
public IPEndPoint GetBackendIPEndpoint()
{
if (!int.TryParse(
Environment.GetEnvironmentVariable(DotnetBackendPortEnvVarName),
out int portNumber))
{
_logger.LogInfo($"'{DotnetBackendPortEnvVarName}' environment variable is not set.");
portNumber = DotnetBackendDebugPort;
}
string ipAddress = Environment.GetEnvironmentVariable(DotnetBackendIPAddressEnvVarName);
if (ipAddress == null)
{
_logger.LogInfo($"'{DotnetBackendIPAddressEnvVarName}' environment variable is not set.");
ipAddress = "127.0.0.1";
}
_logger.LogInfo($"Using IP address {ipAddress} and port {portNumber} for connection.");

return new IPEndPoint(IPAddress.Parse(ipAddress), portNumber);
}

/// <summary>
/// Returns the port number for socket communication between JVM and CLR.
/// </summary>
Expand Down
7 changes: 7 additions & 0 deletions src/csharp/Microsoft.Spark/Services/IConfigurationService.cs
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,8 @@
// The .NET Foundation licenses this file to you under the MIT license.
// See the LICENSE file in the project root for more information.

using System.Net;

namespace Microsoft.Spark.Services
{
/// <summary>
Expand All @@ -14,6 +16,11 @@ internal interface IConfigurationService
/// </summary>
int GetBackendPortNumber();

/// <summary>
/// The IP Endpoint used for communicating with the .NET backend process.
/// </summary>
IPEndPoint GetBackendIPEndpoint();

/// <summary>
/// The IP address used for communicating with the .NET backend process.
/// </summary>
Expand Down
3 changes: 2 additions & 1 deletion src/csharp/Microsoft.Spark/Sql/DataFrame.cs
Original file line number Diff line number Diff line change
Expand Up @@ -886,9 +886,10 @@ public DataStreamWriter WriteStream() =>
private IEnumerable<Row> GetRows(string funcName)
{
(int port, string secret) = GetConnectionInfo(funcName);
IPEndPoint dotnetBackendIPEndpoint = SparkEnvironment.ConfigurationService.GetBackendIPEndpoint();
using (ISocketWrapper socket = SocketFactory.CreateSocket())
{
socket.Connect(IPAddress.Loopback, port, secret);
socket.Connect(dotnetBackendIPEndpoint.Address, port, secret);
foreach (Row row in new RowCollector().Collect(socket))
{
yield return row;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ class DotnetBackend extends Logging {
private[this] var bootstrap: ServerBootstrap = _
private[this] var bossGroup: EventLoopGroup = _

def init(portNumber: Int, ipAddress: String): Int = {
def init(ipAddress: String, portNumber: Int): Int = {
// need at least 3 threads, use 10 here for safety
bossGroup = new NioEventLoopGroup(10)
val workerGroup = bossGroup
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,6 @@ object DotnetRunner extends Logging {
// In debug mode this runner will not launch a .NET process.
val runInDebugMode = settings._1
@volatile var dotnetBackendPortNumber = settings._2
val dotnetBackendIPAddress = settings._3
var dotnetExecutable = ""
var otherArgs: Array[String] = null

Expand Down Expand Up @@ -91,6 +90,7 @@ object DotnetRunner extends Logging {

// Time to wait for DotnetBackend to initialize in seconds.
val backendTimeout = sys.env.getOrElse("DOTNETBACKEND_TIMEOUT", "120").toInt
val dotnetBackendIPAddress = sys.env.getOrElse("DOTNET_SPARK_BACKEND_IP_ADDRESS", "127.0.0.1")

// Launch a DotnetBackend server for the .NET process to connect to; this will let it see our
// Java system properties etc.
Expand All @@ -100,7 +100,7 @@ object DotnetRunner extends Logging {
override def run() {
// need to get back dotnetBackendPortNumber because if the value passed to init is 0
// the port number is dynamically assigned in the backend
dotnetBackendPortNumber = dotnetBackend.init(dotnetBackendPortNumber, dotnetBackendIPAddress)
dotnetBackendPortNumber = dotnetBackend.init(dotnetBackendIPAddress, dotnetBackendPortNumber)
logInfo(s"Port number used by DotnetBackend is $dotnetBackendPortNumber on IP address " +
s"$dotnetBackendIPAddress")
initialized.release()
Expand All @@ -118,7 +118,6 @@ object DotnetRunner extends Logging {
val builder = new ProcessBuilder(processParameters)
val env = builder.environment()
env.put("DOTNETBACKEND_PORT", dotnetBackendPortNumber.toString)
env.put("DOTNETBACKEND_IP_ADDRESS", dotnetBackendIPAddress)

for ((key, value) <- Utils.getSystemProperties if key.startsWith("spark.")) {
env.put(key, value)
Expand Down Expand Up @@ -268,30 +267,19 @@ object DotnetRunner extends Logging {
returnCode
}

private def initializeSettings(args: Array[String]): (Boolean, Int, String) = {
private def initializeSettings(args: Array[String]): (Boolean, Int) = {
val runInDebugMode = (args.length == 1 || args.length == 2) && args(0).equalsIgnoreCase(
"debug")
var portNumber = 0
var dotnetBackendIPAddress = "localhost"
if (runInDebugMode) {
if (args.length == 1) {
portNumber = DEBUG_PORT
} else if (args.length == 2) {
portNumber = Integer.parseInt(args(1))
}
}
else {
try {
var addr = InetAddress.getByName(args(0))
dotnetBackendIPAddress = args(0)
}
catch {
case e: UnknownHostException =>
dotnetBackendIPAddress = "localhost"
}
}

(runInDebugMode, portNumber, dotnetBackendIPAddress)
(runInDebugMode, portNumber)
}

private def logThrowable(throwable: Throwable): Unit =
Expand Down
0