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-25299] Replace ShuffleLocation with BlockManagerId #569

Merged
merged 21 commits into from
Aug 2, 2019
Merged
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

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.spark.api.shuffle;

import org.apache.spark.api.java.Optional;
import org.apache.spark.storage.BlockManagerId;

import java.util.Objects;

Expand All @@ -31,10 +32,10 @@ public class ShuffleBlockInfo {
private final int mapId;
private final int reduceId;
private final long length;
private final Optional<ShuffleLocation> shuffleLocation;
private final Optional<BlockManagerId> shuffleLocation;

public ShuffleBlockInfo(int shuffleId, int mapId, int reduceId, long length,
Optional<ShuffleLocation> shuffleLocation) {
Optional<BlockManagerId> shuffleLocation) {
this.shuffleId = shuffleId;
this.mapId = mapId;
this.reduceId = reduceId;
Expand All @@ -58,7 +59,7 @@ public long getLength() {
return length;
}

public Optional<ShuffleLocation> getShuffleLocation() {
public Optional<BlockManagerId> getShuffleLocation() {
return shuffleLocation;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,4 +30,8 @@ public interface ShuffleDriverComponents {
void cleanupApplication() throws IOException;

void removeShuffleData(int shuffleId, boolean blocking) throws IOException;

default boolean shouldUnregisterOutputOnHostOnFetchFailure() {
return false;
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import org.apache.spark.annotation.Experimental;
import org.apache.spark.api.java.Optional;
import org.apache.spark.storage.BlockManagerId;

/**
* :: Experimental ::
Expand All @@ -32,7 +33,7 @@
public interface ShuffleMapOutputWriter {
ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException;

Optional<MapShuffleLocations> commitAllPartitions() throws IOException;
Optional<BlockManagerId> commitAllPartitions() throws IOException;

void abort(Throwable error) throws IOException;
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.nio.channels.FileChannel;
import javax.annotation.Nullable;

import org.apache.spark.api.java.Optional;
import scala.None$;
import scala.Option;
import scala.Product2;
Expand All @@ -39,8 +40,6 @@
import org.apache.spark.Partitioner;
import org.apache.spark.ShuffleDependency;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.Optional;
import org.apache.spark.api.shuffle.MapShuffleLocations;
import org.apache.spark.api.shuffle.SupportsTransferTo;
import org.apache.spark.api.shuffle.ShuffleMapOutputWriter;
import org.apache.spark.api.shuffle.ShufflePartitionWriter;
Expand Down Expand Up @@ -134,11 +133,8 @@ public void write(Iterator<Product2<K, V>> records) throws IOException {
try {
if (!records.hasNext()) {
partitionLengths = new long[numPartitions];
Optional<MapShuffleLocations> blockLocs = mapOutputWriter.commitAllPartitions();
mapStatus = MapStatus$.MODULE$.apply(
blockManager.shuffleServerId(),
blockLocs.orNull(),
partitionLengths);
Optional<BlockManagerId> location = mapOutputWriter.commitAllPartitions();
mapStatus = MapStatus$.MODULE$.apply(Option.apply(location.orNull()), partitionLengths);
return;
}
final SerializerInstance serInstance = serializer.newInstance();
Expand Down Expand Up @@ -171,11 +167,8 @@ public void write(Iterator<Product2<K, V>> records) throws IOException {
}

partitionLengths = writePartitionedData(mapOutputWriter);
Optional<MapShuffleLocations> mapLocations = mapOutputWriter.commitAllPartitions();
mapStatus = MapStatus$.MODULE$.apply(
blockManager.shuffleServerId(),
mapLocations.orNull(),
partitionLengths);
Optional<BlockManagerId> location = mapOutputWriter.commitAllPartitions();
mapStatus = MapStatus$.MODULE$.apply(Option.apply(location.orNull()), partitionLengths);
} catch (Exception e) {
try {
mapOutputWriter.abort(e);
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import java.nio.channels.FileChannel;
import java.util.Iterator;

import org.apache.spark.api.java.Optional;
import org.apache.spark.storage.BlockManagerId;
import scala.Option;
import scala.Product2;
import scala.collection.JavaConverters;
Expand All @@ -37,8 +39,6 @@

import org.apache.spark.*;
import org.apache.spark.annotation.Private;
import org.apache.spark.api.java.Optional;
import org.apache.spark.api.shuffle.MapShuffleLocations;
import org.apache.spark.api.shuffle.TransferrableWritableByteChannel;
import org.apache.spark.api.shuffle.ShuffleMapOutputWriter;
import org.apache.spark.api.shuffle.ShufflePartitionWriter;
Expand Down Expand Up @@ -219,7 +219,7 @@ void closeAndWriteOutput() throws IOException {
final ShuffleMapOutputWriter mapWriter = shuffleWriteSupport
.createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions());
final long[] partitionLengths;
Optional<MapShuffleLocations> mapLocations;
Optional<BlockManagerId> location;
try {
try {
partitionLengths = mergeSpills(spills, mapWriter);
Expand All @@ -230,7 +230,7 @@ void closeAndWriteOutput() throws IOException {
}
}
}
mapLocations = mapWriter.commitAllPartitions();
location = mapWriter.commitAllPartitions();
} catch (Exception e) {
try {
mapWriter.abort(e);
Expand All @@ -239,10 +239,7 @@ void closeAndWriteOutput() throws IOException {
}
throw e;
}
mapStatus = MapStatus$.MODULE$.apply(
blockManager.shuffleServerId(),
mapLocations.orNull(),
partitionLengths);
mapStatus = MapStatus$.MODULE$.apply(Option.apply(location.orNull()), partitionLengths);
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,21 +24,19 @@
import java.io.OutputStream;
import java.nio.channels.FileChannel;

import org.apache.spark.api.java.Optional;
import org.apache.spark.storage.BlockManagerId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.spark.SparkConf;
import org.apache.spark.api.java.Optional;
import org.apache.spark.api.shuffle.MapShuffleLocations;
import org.apache.spark.api.shuffle.ShuffleMapOutputWriter;
import org.apache.spark.api.shuffle.ShufflePartitionWriter;
import org.apache.spark.api.shuffle.SupportsTransferTo;
import org.apache.spark.api.shuffle.TransferrableWritableByteChannel;
import org.apache.spark.internal.config.package$;
import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations;
import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel;
import org.apache.spark.shuffle.ShuffleWriteMetricsReporter;
import org.apache.spark.storage.BlockManagerId;
import org.apache.spark.shuffle.IndexShuffleBlockResolver;
import org.apache.spark.storage.TimeTrackingOutputStream;
import org.apache.spark.util.Utils;
Expand Down Expand Up @@ -104,11 +102,11 @@ public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOExcep
}

@Override
public Optional<MapShuffleLocations> commitAllPartitions() throws IOException {
public Optional<BlockManagerId> commitAllPartitions() throws IOException {
cleanUp();
File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null;
blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp);
return Optional.of(DefaultMapShuffleLocations.get(shuffleServerId));
return Optional.of(shuffleServerId);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import com.google.common.collect.ImmutableMap;
import org.apache.spark.SparkEnv;
import org.apache.spark.api.shuffle.ShuffleDriverComponents;
import org.apache.spark.internal.config.package$;
import org.apache.spark.storage.BlockManagerMaster;

import java.io.IOException;
Expand All @@ -28,10 +29,15 @@
public class DefaultShuffleDriverComponents implements ShuffleDriverComponents {

private BlockManagerMaster blockManagerMaster;
private boolean shouldUnregisterOutputOnHostOnFetchFailure;

@Override
public Map<String, String> initializeApplication() {
blockManagerMaster = SparkEnv.get().blockManager().master();
this.shouldUnregisterOutputOnHostOnFetchFailure =
SparkEnv.get().blockManager().externalShuffleServiceEnabled()
&& (boolean) SparkEnv.get().conf()
.get(package$.MODULE$.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE());
return ImmutableMap.of();
}

Expand All @@ -46,6 +52,11 @@ public void removeShuffleData(int shuffleId, boolean blocking) throws IOExceptio
blockManagerMaster.removeShuffle(shuffleId, blocking);
}

@Override
public boolean shouldUnregisterOutputOnHostOnFetchFailure() {
return shouldUnregisterOutputOnHostOnFetchFailure;
}

private void checkInitialized() {
if (blockManagerMaster == null) {
throw new IllegalStateException("Driver components must be initialized before using");
Expand Down
Loading