-
Notifications
You must be signed in to change notification settings - Fork 28.3k
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
[SPARK-4307] Initialize FileDescriptor lazily in FileRegion. #3172
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -31,27 +31,27 @@ | |
|
||
import org.apache.spark.network.util.JavaUtils; | ||
import org.apache.spark.network.util.LimitedInputStream; | ||
import org.apache.spark.network.util.SystemPropertyConfigProvider; | ||
import org.apache.spark.network.util.TransportConf; | ||
|
||
/** | ||
* A {@link ManagedBuffer} backed by a segment in a file. | ||
*/ | ||
public final class FileSegmentManagedBuffer extends ManagedBuffer { | ||
|
||
/** | ||
* Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). | ||
* Avoid unless there's a good reason not to. | ||
*/ | ||
// TODO: Make this configurable | ||
private static final long MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; | ||
|
||
private final File file; | ||
private final long offset; | ||
private final long length; | ||
private final TransportConf conf; | ||
|
||
public FileSegmentManagedBuffer(File file, long offset, long length) { | ||
public FileSegmentManagedBuffer(File file, long offset, long length, TransportConf conf) { | ||
this.file = file; | ||
this.offset = offset; | ||
this.length = length; | ||
this.conf = conf; | ||
} | ||
|
||
public FileSegmentManagedBuffer(File file, long offset, long length) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Please add a comment that the other constructor is preferred (this creates an extra object, for instance, and gets configuration from a random place). |
||
this(file, offset, length, new TransportConf(new SystemPropertyConfigProvider())); | ||
} | ||
|
||
@Override | ||
|
@@ -65,7 +65,7 @@ public ByteBuffer nioByteBuffer() throws IOException { | |
try { | ||
channel = new RandomAccessFile(file, "r").getChannel(); | ||
// Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. | ||
if (length < MIN_MEMORY_MAP_BYTES) { | ||
if (length < conf.memoryMapBytes()) { | ||
ByteBuffer buf = ByteBuffer.allocate((int) length); | ||
channel.position(offset); | ||
while (buf.remaining() != 0) { | ||
|
@@ -134,8 +134,12 @@ public ManagedBuffer release() { | |
|
||
@Override | ||
public Object convertToNetty() throws IOException { | ||
FileChannel fileChannel = new FileInputStream(file).getChannel(); | ||
return new DefaultFileRegion(fileChannel, offset, length); | ||
if (conf.lazyFileDescriptor()) { | ||
return new LazyFileRegion(file, offset, length); | ||
} else { | ||
FileChannel fileChannel = new FileInputStream(file).getChannel(); | ||
return new DefaultFileRegion(fileChannel, offset, length); | ||
} | ||
} | ||
|
||
public File getFile() { return file; } | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,108 @@ | ||
/* | ||
* 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.buffer; | ||
|
||
import java.io.FileInputStream; | ||
import java.io.File; | ||
import java.io.IOException; | ||
import java.nio.channels.FileChannel; | ||
import java.nio.channels.WritableByteChannel; | ||
|
||
import com.google.common.base.Objects; | ||
import io.netty.channel.FileRegion; | ||
import io.netty.util.AbstractReferenceCounted; | ||
|
||
import org.apache.spark.network.util.JavaUtils; | ||
|
||
/** | ||
* A FileRegion implementation that only creates the file descriptor when the region is being | ||
* transferred. This cannot be used with Epoll because there is no native support for it. | ||
*/ | ||
public final class LazyFileRegion extends AbstractReferenceCounted implements FileRegion { | ||
|
||
private final File file; | ||
private final long position; | ||
private final long count; | ||
|
||
private FileChannel channel; | ||
|
||
private long numBytesTransferred = 0L; | ||
|
||
/** | ||
* @param file file to transfer. | ||
* @param position start position for the transfer. | ||
* @param count number of bytes to transfer starting from position. | ||
*/ | ||
public LazyFileRegion(File file, long position, long count) { | ||
this.file = file; | ||
this.position = position; | ||
this.count = count; | ||
} | ||
|
||
@Override | ||
protected void deallocate() { | ||
JavaUtils.closeQuietly(channel); | ||
} | ||
|
||
@Override | ||
public long position() { | ||
return position; | ||
} | ||
|
||
@Override | ||
public long transfered() { | ||
return numBytesTransferred; | ||
} | ||
|
||
@Override | ||
public long count() { | ||
return count; | ||
} | ||
|
||
@Override | ||
public long transferTo(WritableByteChannel target, long position) throws IOException { | ||
if (channel == null) { | ||
channel = new FileInputStream(file).getChannel(); | ||
} | ||
|
||
long count = this.count - position; | ||
if (count < 0 || position < 0) { | ||
throw new IllegalArgumentException( | ||
"position out of range: " + position + " (expected: 0 - " + (count - 1) + ')'); | ||
} | ||
|
||
if (count == 0) { | ||
return 0L; | ||
} | ||
|
||
long written = channel.transferTo(this.position + position, count, target); | ||
if (written > 0) { | ||
numBytesTransferred += written; | ||
} | ||
return written; | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return Objects.toStringHelper(this) | ||
.add("file", file) | ||
.add("position", position) | ||
.add("count", count) | ||
.toString(); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,13 +30,15 @@ | |
import com.google.common.annotations.VisibleForTesting; | ||
import com.google.common.base.Objects; | ||
import com.google.common.collect.Maps; | ||
import org.apache.spark.network.util.SystemPropertyConfigProvider; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import org.apache.spark.network.buffer.FileSegmentManagedBuffer; | ||
import org.apache.spark.network.buffer.ManagedBuffer; | ||
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; | ||
import org.apache.spark.network.util.JavaUtils; | ||
import org.apache.spark.network.util.TransportConf; | ||
|
||
/** | ||
* Manages converting shuffle BlockIds into physical segments of local files, from a process outside | ||
|
@@ -56,16 +58,24 @@ public class ExternalShuffleBlockManager { | |
// Single-threaded Java executor used to perform expensive recursive directory deletion. | ||
private final Executor directoryCleaner; | ||
|
||
public ExternalShuffleBlockManager() { | ||
private final TransportConf conf; | ||
|
||
public ExternalShuffleBlockManager(TransportConf conf) { | ||
// TODO: Give this thread a name. | ||
this(Executors.newSingleThreadExecutor()); | ||
this(Executors.newSingleThreadExecutor(), conf); | ||
} | ||
|
||
// Allows tests to have more control over when directories are cleaned up. | ||
@VisibleForTesting | ||
ExternalShuffleBlockManager(Executor directoryCleaner) { | ||
ExternalShuffleBlockManager(Executor directoryCleaner, TransportConf conf) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: please make conf first parameter to be consistent with other classes |
||
this.executors = Maps.newConcurrentMap(); | ||
this.directoryCleaner = directoryCleaner; | ||
this.conf = conf; | ||
} | ||
|
||
@VisibleForTesting | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we probably don't need 2 VisibleForTesting constructors, do we? If someone wants to provide an explicit Executor, they can damn well create their own TransportConf. This is only done in a single unit test anyway. |
||
ExternalShuffleBlockManager(Executor directoryCleaner) { | ||
this(directoryCleaner, new TransportConf(new SystemPropertyConfigProvider())); | ||
} | ||
|
||
/** Registers a new Executor with all the configuration we need to find its shuffle files. */ | ||
|
@@ -167,7 +177,7 @@ private void deleteExecutorDirs(String[] dirs) { | |
// TODO: Support consolidated hash shuffle files | ||
private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, String blockId) { | ||
File shuffleFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); | ||
return new FileSegmentManagedBuffer(shuffleFile, 0, shuffleFile.length()); | ||
return new FileSegmentManagedBuffer(shuffleFile, 0, shuffleFile.length(), conf); | ||
} | ||
|
||
/** | ||
|
@@ -190,7 +200,8 @@ private ManagedBuffer getSortBasedShuffleBlockData( | |
getFile(executor.localDirs, executor.subDirsPerLocalDir, | ||
"shuffle_" + shuffleId + "_" + mapId + "_0.data"), | ||
offset, | ||
nextOffset - offset); | ||
nextOffset - offset, | ||
conf); | ||
} catch (IOException e) { | ||
throw new RuntimeException("Failed to open file: " + indexFile, e); | ||
} finally { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: import order