|
| 1 | +/* |
| 2 | + * Licensed to the .NET Foundation under one or more agreements. |
| 3 | + * The .NET Foundation licenses this file to you under the MIT license. |
| 4 | + * See the LICENSE file in the project root for more information. |
| 5 | + */ |
| 6 | + |
| 7 | +package org.apache.spark.api.dotnet |
| 8 | + |
| 9 | +import java.io.{ByteArrayOutputStream, Closeable, DataInputStream, DataOutputStream} |
| 10 | +import java.net.Socket |
| 11 | + |
| 12 | +import org.apache.spark.internal.Logging |
| 13 | + |
| 14 | +/** |
| 15 | + * CallbackConnection is used to process the callback communication |
| 16 | + * between the JVM and Dotnet. It uses a TCP socket to communicate with |
| 17 | + * the Dotnet CallbackServer and the socket is expected to be reused. |
| 18 | + * @param address The address of the Dotnet CallbackServer |
| 19 | + * @param port The port of the Dotnet CallbackServer |
| 20 | + */ |
| 21 | +class CallbackConnection(serDe: SerDe, address: String, port: Int) extends Logging { |
| 22 | + private[this] val socket: Socket = new Socket(address, port) |
| 23 | + private[this] val inputStream: DataInputStream = new DataInputStream(socket.getInputStream) |
| 24 | + private[this] val outputStream: DataOutputStream = new DataOutputStream(socket.getOutputStream) |
| 25 | + |
| 26 | + def send( |
| 27 | + callbackId: Int, |
| 28 | + writeBody: (DataOutputStream, SerDe) => Unit): Unit = { |
| 29 | + logInfo(s"Calling callback [callback id = $callbackId] ...") |
| 30 | + |
| 31 | + try { |
| 32 | + serDe.writeInt(outputStream, CallbackFlags.CALLBACK) |
| 33 | + serDe.writeInt(outputStream, callbackId) |
| 34 | + |
| 35 | + val byteArrayOutputStream = new ByteArrayOutputStream() |
| 36 | + writeBody(new DataOutputStream(byteArrayOutputStream), serDe) |
| 37 | + serDe.writeInt(outputStream, byteArrayOutputStream.size) |
| 38 | + byteArrayOutputStream.writeTo(outputStream); |
| 39 | + } catch { |
| 40 | + case e: Exception => { |
| 41 | + throw new Exception("Error writing to stream.", e) |
| 42 | + } |
| 43 | + } |
| 44 | + |
| 45 | + logInfo(s"Signaling END_OF_STREAM.") |
| 46 | + try { |
| 47 | + serDe.writeInt(outputStream, CallbackFlags.END_OF_STREAM) |
| 48 | + outputStream.flush() |
| 49 | + |
| 50 | + val endOfStreamResponse = readFlag(inputStream) |
| 51 | + endOfStreamResponse match { |
| 52 | + case CallbackFlags.END_OF_STREAM => |
| 53 | + logInfo(s"Received END_OF_STREAM signal. Calling callback [callback id = $callbackId] successful.") |
| 54 | + case _ => { |
| 55 | + throw new Exception(s"Error verifying end of stream. Expected: ${CallbackFlags.END_OF_STREAM}, " + |
| 56 | + s"Received: $endOfStreamResponse") |
| 57 | + } |
| 58 | + } |
| 59 | + } catch { |
| 60 | + case e: Exception => { |
| 61 | + throw new Exception("Error while verifying end of stream.", e) |
| 62 | + } |
| 63 | + } |
| 64 | + } |
| 65 | + |
| 66 | + def close(): Unit = { |
| 67 | + try { |
| 68 | + serDe.writeInt(outputStream, CallbackFlags.CLOSE) |
| 69 | + outputStream.flush() |
| 70 | + } catch { |
| 71 | + case e: Exception => logInfo("Unable to send close to .NET callback server.", e) |
| 72 | + } |
| 73 | + |
| 74 | + close(socket) |
| 75 | + close(outputStream) |
| 76 | + close(inputStream) |
| 77 | + } |
| 78 | + |
| 79 | + private def close(s: Socket): Unit = { |
| 80 | + try { |
| 81 | + assert(s != null) |
| 82 | + s.close() |
| 83 | + } catch { |
| 84 | + case e: Exception => logInfo("Unable to close socket.", e) |
| 85 | + } |
| 86 | + } |
| 87 | + |
| 88 | + private def close(c: Closeable): Unit = { |
| 89 | + try { |
| 90 | + assert(c != null) |
| 91 | + c.close() |
| 92 | + } catch { |
| 93 | + case e: Exception => logInfo("Unable to close closeable.", e) |
| 94 | + } |
| 95 | + } |
| 96 | + |
| 97 | + private def readFlag(inputStream: DataInputStream): Int = { |
| 98 | + val callbackFlag = serDe.readInt(inputStream) |
| 99 | + if (callbackFlag == CallbackFlags.DOTNET_EXCEPTION_THROWN) { |
| 100 | + val exceptionMessage = serDe.readString(inputStream) |
| 101 | + throw new DotnetException(exceptionMessage) |
| 102 | + } |
| 103 | + callbackFlag |
| 104 | + } |
| 105 | + |
| 106 | + private object CallbackFlags { |
| 107 | + val CLOSE: Int = -1 |
| 108 | + val CALLBACK: Int = -2 |
| 109 | + val DOTNET_EXCEPTION_THROWN: Int = -3 |
| 110 | + val END_OF_STREAM: Int = -4 |
| 111 | + } |
| 112 | +} |
0 commit comments