Skip to content

[SPARK-11956] [core] Fix a few bugs in network lib-based file transfer. #9941

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

Closed
wants to merge 1 commit into from
Closed
Show file tree
Hide file tree
Changes from all commits
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
19 changes: 14 additions & 5 deletions core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import javax.annotation.Nullable

import scala.concurrent.{Future, Promise}
import scala.reflect.ClassTag
import scala.util.{DynamicVariable, Failure, Success}
import scala.util.{DynamicVariable, Failure, Success, Try}
import scala.util.control.NonFatal

import org.apache.spark.{Logging, SecurityManager, SparkConf}
Expand Down Expand Up @@ -368,13 +368,22 @@ private[netty] class NettyRpcEnv(

@volatile private var error: Throwable = _

def setError(e: Throwable): Unit = error = e
def setError(e: Throwable): Unit = {
error = e
source.close()
}

override def read(dst: ByteBuffer): Int = {
if (error != null) {
throw error
val result = if (error == null) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it same as the previous codes?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's similar, but slightly different. Because of source.close() in L373, I'm giving preference to the error that caused the channel to be closed; the previous code would instead throw a ChannelClosedException or something.

Try(source.read(dst))
} else {
Failure(error)
}

result match {
case Success(bytesRead) => bytesRead
case Failure(error) => throw error
}
source.read(dst)
}

override def close(): Unit = source.close()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ private[netty] class NettyStreamManager(rpcEnv: NettyRpcEnv)
case _ => throw new IllegalArgumentException(s"Invalid file type: $ftype")
}

require(file != null, s"File not found: $streamId")
require(file != null && file.isFile(), s"File not found: $streamId")
new FileSegmentManagedBuffer(rpcEnv.transportConf, file, 0, file.length())
}

Expand Down
27 changes: 20 additions & 7 deletions core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -729,23 +729,36 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll {
val tempDir = Utils.createTempDir()
val file = new File(tempDir, "file")
Files.write(UUID.randomUUID().toString(), file, UTF_8)
val empty = new File(tempDir, "empty")
Files.write("", empty, UTF_8);
val jar = new File(tempDir, "jar")
Files.write(UUID.randomUUID().toString(), jar, UTF_8)

val fileUri = env.fileServer.addFile(file)
val emptyUri = env.fileServer.addFile(empty)
val jarUri = env.fileServer.addJar(jar)

val destDir = Utils.createTempDir()
val destFile = new File(destDir, file.getName())
val destJar = new File(destDir, jar.getName())

val sm = new SecurityManager(conf)
val hc = SparkHadoopUtil.get.conf
Utils.fetchFile(fileUri, destDir, conf, sm, hc, 0L, false)
Utils.fetchFile(jarUri, destDir, conf, sm, hc, 0L, false)

assert(Files.equal(file, destFile))
assert(Files.equal(jar, destJar))
val files = Seq(
(file, fileUri),
(empty, emptyUri),
(jar, jarUri))
files.foreach { case (f, uri) =>
val destFile = new File(destDir, f.getName())
Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false)
assert(Files.equal(f, destFile))
}

// Try to download files that do not exist.
Seq("files", "jars").foreach { root =>
intercept[Exception] {
val uri = env.address.toSparkURL + s"/$root/doesNotExist"
Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false)
}
}
}

}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -185,16 +185,24 @@ public void handle(ResponseMessage message) {
StreamResponse resp = (StreamResponse) message;
StreamCallback callback = streamCallbacks.poll();
if (callback != null) {
StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
callback);
try {
TransportFrameDecoder frameDecoder = (TransportFrameDecoder)
channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME);
frameDecoder.setInterceptor(interceptor);
streamActive = true;
} catch (Exception e) {
logger.error("Error installing stream handler.", e);
deactivateStream();
if (resp.byteCount > 0) {
StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
callback);
try {
TransportFrameDecoder frameDecoder = (TransportFrameDecoder)
channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME);
frameDecoder.setInterceptor(interceptor);
streamActive = true;
} catch (Exception e) {
logger.error("Error installing stream handler.", e);
deactivateStream();
}
} else {
try {
callback.onComplete(resp.streamId);
} catch (Exception e) {
logger.warn("Error in stream handler onComplete().", e);
}
}
} else {
logger.error("Could not find callback for StreamResponse.");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,13 +51,14 @@
import org.apache.spark.network.util.TransportConf;

public class StreamSuite {
private static final String[] STREAMS = { "largeBuffer", "smallBuffer", "file" };
private static final String[] STREAMS = { "largeBuffer", "smallBuffer", "emptyBuffer", "file" };

private static TransportServer server;
private static TransportClientFactory clientFactory;
private static File testFile;
private static File tempDir;

private static ByteBuffer emptyBuffer;
private static ByteBuffer smallBuffer;
private static ByteBuffer largeBuffer;

Expand All @@ -73,6 +74,7 @@ private static ByteBuffer createBuffer(int bufSize) {
@BeforeClass
public static void setUp() throws Exception {
tempDir = Files.createTempDir();
emptyBuffer = createBuffer(0);
smallBuffer = createBuffer(100);
largeBuffer = createBuffer(100000);

Expand Down Expand Up @@ -103,6 +105,8 @@ public ManagedBuffer openStream(String streamId) {
return new NioManagedBuffer(largeBuffer);
case "smallBuffer":
return new NioManagedBuffer(smallBuffer);
case "emptyBuffer":
return new NioManagedBuffer(emptyBuffer);
case "file":
return new FileSegmentManagedBuffer(conf, testFile, 0, testFile.length());
default:
Expand Down Expand Up @@ -138,6 +142,18 @@ public static void tearDown() {
}
}

@Test
public void testZeroLengthStream() throws Throwable {
TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
try {
StreamTask task = new StreamTask(client, "emptyBuffer", TimeUnit.SECONDS.toMillis(5));
task.run();
task.check();
} finally {
client.close();
}
}

@Test
public void testSingleStream() throws Throwable {
TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort());
Expand Down Expand Up @@ -226,6 +242,11 @@ public void run() {
outFile = File.createTempFile("data", ".tmp", tempDir);
out = new FileOutputStream(outFile);
break;
case "emptyBuffer":
baos = new ByteArrayOutputStream();
out = baos;
srcBuffer = emptyBuffer;
break;
default:
throw new IllegalArgumentException(streamId);
}
Expand Down