Skip to content
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-16817][CORE][WIP] Use Alluxio to improve stability of shuffle by replication of shuffle data #22005

Closed
wants to merge 4 commits 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
151 changes: 151 additions & 0 deletions core/src/main/java/org/apache/spark/AlluxioManagedBuffer.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,151 @@
/*
* 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;


import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;


import com.google.common.base.Objects;
import com.google.common.io.ByteStreams;

import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.network.TransportContext;
import org.apache.spark.network.buffer.ManagedBuffer;
import org.apache.spark.network.util.JavaUtils;
import org.apache.spark.network.util.LimitedInputStream;
import org.apache.spark.network.util.TransportConf;
import org.apache.spark.storage.ExternalBlockManager;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import static java.lang.Math.toIntExact;

/**
* A {@link ManagedBuffer} backed by a segment in a file.
*/
public final class AlluxioManagedBuffer extends ManagedBuffer {
private static final Logger logger = LoggerFactory.getLogger(TransportContext.class);

private final TransportConf conf;
private final Path file;
private final long offset;
private final long length;
private final ExternalBlockManager blockManager;

public AlluxioManagedBuffer(TransportConf conf, Path file, long offset, long length, ExternalBlockManager blockManager) {
this.conf = conf;
this.file = file;
this.offset = offset;
this.length = length;
this.blockManager = blockManager;
}

@Override
public long size() {
return length;
}

@Override
public ByteBuffer nioByteBuffer() throws IOException {
FileSystem fs = blockManager.fs();
if (!fs.exists(file)) {
return ByteBuffer.wrap(new byte[0]);
} else {
long size = fs.getFileStatus(file).getLen();
if (size == 0) {
return ByteBuffer.wrap(new byte[0]);
} else {
InputStream input = fs.open(file);
try {
byte[] buffer = new byte[toIntExact(size)];
ByteStreams.readFully(input, buffer);
return ByteBuffer.wrap(buffer,toIntExact(offset),toIntExact(length));
} catch (IOException e){
logger.info("Test-log: Failed to get bytes of block $blockId from Alluxio", e);
return ByteBuffer.wrap(new byte[0]);
} finally {
input.close();
}
}
}
}

@Override
public InputStream createInputStream() throws IOException {

logger.info("Test-log: use function createInputStream");
InputStream is = blockManager.createInputStream(file).get();
logger.info("Test-log: create inputStream successfully");
try {

ByteStreams.skipFully(is, offset);
logger.info("Test-log: read data from inputstream from offset " + offset + " and size is " + length );
return new LimitedInputStream(is, length);
} catch (IOException e) {
try {
if (is != null) {
long size = blockManager.fs().getFileStatus(file).getLen();
throw new IOException("Error in reading " + this + " (actual file length " + size + ")",
e);
}
} catch (IOException ignored) {
// ignore
} finally {
JavaUtils.closeQuietly(is);
}
throw new IOException("Error in opening " + this, e);
} catch (RuntimeException e) {
JavaUtils.closeQuietly(is);
throw e;
}
}

@Override
public ManagedBuffer retain() {
return this;
}

@Override
public ManagedBuffer release() {
return this;
}

@Override
public Object convertToNetty() throws IOException {
return new Object();
}

public Path getFile() { return file; }

public long getOffset() { return offset; }

public long getLength() { return length; }

@Override
public String toString() {
return Objects.toStringHelper(this)
.add("file", file)
.add("offset", offset)
.add("length", length)
.toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,7 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> {
private FileSegment[] partitionWriterSegments;
@Nullable private MapStatus mapStatus;
private long[] partitionLengths;
private boolean useAlluxio;

/**
* Are we in the process of stopping? Because map tasks can call stop() with success = true
Expand All @@ -108,6 +109,7 @@ final class BypassMergeSortShuffleWriter<K, V> extends ShuffleWriter<K, V> {
// Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided
this.fileBufferSize = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024;
this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true);
this.useAlluxio = conf.getBoolean("spark.alluxio.shuffle.enabled",false);
this.blockManager = blockManager;
final ShuffleDependency<K, V, V> dep = handle.dependency();
this.mapId = mapId;
Expand Down Expand Up @@ -162,6 +164,11 @@ public void write(Iterator<Product2<K, V>> records) throws IOException {
try {
partitionLengths = writePartitionedFile(tmp);
shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp);
if(useAlluxio) {
logger.info("shuffle block id " + new ShuffleDataBlockId(shuffleId, mapId, 0).name() + "\nfile path is "
+ output.getPath() + "\nfile name is " + output.getName() + "file size is " + output.length());
blockManager.externalBlockStore().externalBlockManager().get().putFile(shuffleId, new ShuffleDataBlockId(shuffleId, mapId, 0), output);
}
} finally {
if (tmp.exists() && !tmp.delete()) {
logger.error("Error while deleting temp file {}", tmp.getAbsolutePath());
Expand Down
3 changes: 3 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -503,6 +503,9 @@ class SparkContext(config: SparkConf) extends Logging {
_applicationId = _taskScheduler.applicationId()
_applicationAttemptId = taskScheduler.applicationAttemptId()
_conf.set("spark.app.id", _applicationId)
if (_conf.getBoolean("spark.alluxio.shuffle.enabled", false)) {
_env.blockManager.master.init()
}
if (_conf.getBoolean("spark.ui.reverseProxy", false)) {
System.setProperty("spark.ui.proxyBase", "/proxy/" + _applicationId)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -368,8 +368,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
}

if (shouldDisable) {
logInfo(s"Disabling executor $executorId.")
scheduler.executorLost(executorId, LossReasonPending)
if (SparkEnv.get.conf.getBoolean("spark.alluxio.shuffle.enabled", false)) {
scheduler.executorLost(executorId, ExecutorKilled)
} else {
scheduler.executorLost(executorId, LossReasonPending)
}
}

shouldDisable
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,10 @@ private[spark] class IndexShuffleBlockResolver(
if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) {
throw new IOException("fail to rename file " + dataTmp + " to " + dataFile)
}
if (SparkEnv.get.conf.getBoolean("spark.alluxio.shuffle.enabled", false)) {
blockManager.externalBlockStore.externalBlockManager.get.putFile(shuffleId,
ShuffleIndexBlockId(shuffleId, mapId, NOOP_REDUCE_ID), indexFile)
}
}
}
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import org.apache.spark._
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler.MapStatus
import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter}
import org.apache.spark.storage.ShuffleBlockId
import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId}
import org.apache.spark.util.Utils
import org.apache.spark.util.collection.ExternalSorter

Expand Down Expand Up @@ -70,6 +70,13 @@ private[spark] class SortShuffleWriter[K, V, C](
val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)
val partitionLengths = sorter.writePartitionedFile(blockId, tmp)
shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp)
if (SparkEnv.get.conf.getBoolean("spark.alluxio.shuffle.enabled", false)) {
if (output.length() != 0) {
blockManager.externalBlockStore.externalBlockManager.get.putFile(
dep.shuffleId, ShuffleDataBlockId(dep.shuffleId, mapId,
IndexShuffleBlockResolver.NOOP_REDUCE_ID), output)
}
}
mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths)
} finally {
if (tmp.exists() && !tmp.delete()) {
Expand Down
Loading