forked from apache/spark
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-2936] Migrate Netty network module from Java to Scala
The Netty network module was originally written when Scala 2.9.x had a bug that prevents a pure Scala implementation, and a subset of the files were done in Java. We have since upgraded to Scala 2.10, and can migrate all Java files now to Scala. netty/netty#781 mesos/spark#522 Author: Reynold Xin <[email protected]> Closes apache#1865 from rxin/netty and squashes the following commits: 332422f [Reynold Xin] Code review feedback ca9eeee [Reynold Xin] Minor update. 7f1434b [Reynold Xin] [SPARK-2936] Migrate Netty network module from Java to Scala
- Loading branch information
Showing
12 changed files
with
292 additions
and
364 deletions.
There are no files selected for viewing
100 changes: 0 additions & 100 deletions
100
core/src/main/java/org/apache/spark/network/netty/FileClient.java
This file was deleted.
Oops, something went wrong.
111 changes: 0 additions & 111 deletions
111
core/src/main/java/org/apache/spark/network/netty/FileServer.java
This file was deleted.
Oops, something went wrong.
83 changes: 0 additions & 83 deletions
83
core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java
This file was deleted.
Oops, something went wrong.
85 changes: 85 additions & 0 deletions
85
core/src/main/scala/org/apache/spark/network/netty/FileClient.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,85 @@ | ||
/* | ||
* 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.spark.network.netty | ||
|
||
import java.util.concurrent.TimeUnit | ||
|
||
import io.netty.bootstrap.Bootstrap | ||
import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} | ||
import io.netty.channel.oio.OioEventLoopGroup | ||
import io.netty.channel.socket.oio.OioSocketChannel | ||
|
||
import org.apache.spark.Logging | ||
|
||
class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { | ||
|
||
private var channel: Channel = _ | ||
private var bootstrap: Bootstrap = _ | ||
private var group: EventLoopGroup = _ | ||
private val sendTimeout = 60 | ||
|
||
def init(): Unit = { | ||
group = new OioEventLoopGroup | ||
bootstrap = new Bootstrap | ||
bootstrap.group(group) | ||
.channel(classOf[OioSocketChannel]) | ||
.option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) | ||
.option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) | ||
.option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) | ||
.handler(new FileClientChannelInitializer(handler)) | ||
} | ||
|
||
def connect(host: String, port: Int) { | ||
try { | ||
channel = bootstrap.connect(host, port).sync().channel() | ||
} catch { | ||
case e: InterruptedException => | ||
logWarning("FileClient interrupted while trying to connect", e) | ||
close() | ||
} | ||
} | ||
|
||
def waitForClose(): Unit = { | ||
try { | ||
channel.closeFuture.sync() | ||
} catch { | ||
case e: InterruptedException => | ||
logWarning("FileClient interrupted", e) | ||
} | ||
} | ||
|
||
def sendRequest(file: String): Unit = { | ||
try { | ||
val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) | ||
if (!bSent) { | ||
throw new RuntimeException("Failed to send") | ||
} | ||
} catch { | ||
case e: InterruptedException => | ||
logError("Error", e) | ||
} | ||
} | ||
|
||
def close(): Unit = { | ||
if (group != null) { | ||
group.shutdownGracefully() | ||
group = null | ||
bootstrap = null | ||
} | ||
} | ||
} |
Oops, something went wrong.