Skip to content

Commit

Permalink
[SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API
Browse files Browse the repository at this point in the history
## What changes were proposed in this pull request?

As part of the shuffle storage API proposed in SPARK-25299, this introduces an API for persisting shuffle data in arbitrary storage systems.

This patch introduces several concepts:
* `ShuffleDataIO`, which is the root of the entire plugin tree that will be proposed over the course of the shuffle API project.
* `ShuffleExecutorComponents` - the subset of plugins for managing shuffle-related components for each executor. This will in turn instantiate shuffle readers and writers.
* `ShuffleMapOutputWriter` interface - instantiated once per map task. This provides child `ShufflePartitionWriter` instances for persisting the bytes for each partition in the map task.

The default implementation of these plugins exactly mirror what was done by the existing shuffle writing code - namely, writing the data to local disk and writing an index file. We leverage the APIs in the `BypassMergeSortShuffleWriter` only. Follow-up PRs will use the APIs in `SortShuffleWriter` and `UnsafeShuffleWriter`, but are left as future work to minimize the review surface area.

## How was this patch tested?

New unit tests were added. Micro-benchmarks indicate there's no slowdown in the affected code paths.

Closes #25007 from mccheah/spark-shuffle-writer-refactor.

Lead-authored-by: mcheah <mcheah@palantir.com>
Co-authored-by: mccheah <mcheah@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
  • Loading branch information
mccheah authored and Marcelo Vanzin committed Jul 30, 2019
1 parent 121f933 commit abef84a
Show file tree
Hide file tree
Showing 15 changed files with 1,087 additions and 147 deletions.
49 changes: 49 additions & 0 deletions core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
/*
* 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.shuffle.api;

import org.apache.spark.annotation.Private;

/**
* :: Private ::
* An interface for plugging in modules for storing and reading temporary shuffle data.
* <p>
* This is the root of a plugin system for storing shuffle bytes to arbitrary storage
* backends in the sort-based shuffle algorithm implemented by the
* {@link org.apache.spark.shuffle.sort.SortShuffleManager}. If another shuffle algorithm is
* needed instead of sort-based shuffle, one should implement
* {@link org.apache.spark.shuffle.ShuffleManager} instead.
* <p>
* A single instance of this module is loaded per process in the Spark application.
* The default implementation reads and writes shuffle data from the local disks of
* the executor, and is the implementation of shuffle file storage that has remained
* consistent throughout most of Spark's history.
* <p>
* Alternative implementations of shuffle data storage can be loaded via setting
* <code>spark.shuffle.sort.io.plugin.class</code>.
* @since 3.0.0
*/
@Private
public interface ShuffleDataIO {

/**
* Called once on executor processes to bootstrap the shuffle data storage modules that
* are only invoked on the executors.
*/
ShuffleExecutorComponents executor();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.shuffle.api;

import java.io.IOException;

import org.apache.spark.annotation.Private;

/**
* :: Private ::
* An interface for building shuffle support for Executors.
*
* @since 3.0.0
*/
@Private
public interface ShuffleExecutorComponents {

/**
* Called once per executor to bootstrap this module with state that is specific to
* that executor, specifically the application ID and executor ID.
*/
void initializeExecutor(String appId, String execId);

/**
* Called once per map task to create a writer that will be responsible for persisting all the
* partitioned bytes written by that map task.
* @param shuffleId Unique identifier for the shuffle the map task is a part of
* @param mapId Within the shuffle, the identifier of the map task
* @param mapTaskAttemptId Identifier of the task attempt. Multiple attempts of the same map task
* with the same (shuffleId, mapId) pair can be distinguished by the
* different values of mapTaskAttemptId.
* @param numPartitions The number of partitions that will be written by the map task. Some of
* these partitions may be empty.
*/
ShuffleMapOutputWriter createMapOutputWriter(
int shuffleId,
int mapId,
long mapTaskAttemptId,
int numPartitions) throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.shuffle.api;

import java.io.IOException;

import org.apache.spark.annotation.Private;

/**
* :: Private ::
* A top-level writer that returns child writers for persisting the output of a map task,
* and then commits all of the writes as one atomic operation.
*
* @since 3.0.0
*/
@Private
public interface ShuffleMapOutputWriter {

/**
* Creates a writer that can open an output stream to persist bytes targeted for a given reduce
* partition id.
* <p>
* The chunk corresponds to bytes in the given reduce partition. This will not be called twice
* for the same partition within any given map task. The partition identifier will be in the
* range of precisely 0 (inclusive) to numPartitions (exclusive), where numPartitions was
* provided upon the creation of this map output writer via
* {@link ShuffleExecutorComponents#createMapOutputWriter(int, int, long, int)}.
* <p>
* Calls to this method will be invoked with monotonically increasing reducePartitionIds; each
* call to this method will be called with a reducePartitionId that is strictly greater than
* the reducePartitionIds given to any previous call to this method. This method is not
* guaranteed to be called for every partition id in the above described range. In particular,
* no guarantees are made as to whether or not this method will be called for empty partitions.
*/
ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws IOException;

/**
* Commits the writes done by all partition writers returned by all calls to this object's
* {@link #getPartitionWriter(int)}.
* <p>
* This should ensure that the writes conducted by this module's partition writers are
* available to downstream reduce tasks. If this method throws any exception, this module's
* {@link #abort(Throwable)} method will be invoked before propagating the exception.
* <p>
* This can also close any resources and clean up temporary state if necessary.
*/
void commitAllPartitions() throws IOException;

/**
* Abort all of the writes done by any writers returned by {@link #getPartitionWriter(int)}.
* <p>
* This should invalidate the results of writing bytes. This can also close any resources and
* clean up temporary state if necessary.
*/
void abort(Throwable error) throws IOException;
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,98 @@
/*
* 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.shuffle.api;

import java.io.IOException;
import java.util.Optional;
import java.io.OutputStream;

import org.apache.spark.annotation.Private;

/**
* :: Private ::
* An interface for opening streams to persist partition bytes to a backing data store.
* <p>
* This writer stores bytes for one (mapper, reducer) pair, corresponding to one shuffle
* block.
*
* @since 3.0.0
*/
@Private
public interface ShufflePartitionWriter {

/**
* Open and return an {@link OutputStream} that can write bytes to the underlying
* data store.
* <p>
* This method will only be called once on this partition writer in the map task, to write the
* bytes to the partition. The output stream will only be used to write the bytes for this
* partition. The map task closes this output stream upon writing all the bytes for this
* block, or if the write fails for any reason.
* <p>
* Implementations that intend on combining the bytes for all the partitions written by this
* map task should reuse the same OutputStream instance across all the partition writers provided
* by the parent {@link ShuffleMapOutputWriter}. If one does so, ensure that
* {@link OutputStream#close()} does not close the resource, since it will be reused across
* partition writes. The underlying resources should be cleaned up in
* {@link ShuffleMapOutputWriter#commitAllPartitions()} and
* {@link ShuffleMapOutputWriter#abort(Throwable)}.
*/
OutputStream openStream() throws IOException;

/**
* Opens and returns a {@link WritableByteChannelWrapper} for transferring bytes from
* input byte channels to the underlying shuffle data store.
* <p>
* This method will only be called once on this partition writer in the map task, to write the
* bytes to the partition. The channel will only be used to write the bytes for this
* partition. The map task closes this channel upon writing all the bytes for this
* block, or if the write fails for any reason.
* <p>
* Implementations that intend on combining the bytes for all the partitions written by this
* map task should reuse the same channel instance across all the partition writers provided
* by the parent {@link ShuffleMapOutputWriter}. If one does so, ensure that
* {@link WritableByteChannelWrapper#close()} does not close the resource, since the channel
* will be reused across partition writes. The underlying resources should be cleaned up in
* {@link ShuffleMapOutputWriter#commitAllPartitions()} and
* {@link ShuffleMapOutputWriter#abort(Throwable)}.
* <p>
* This method is primarily for advanced optimizations where bytes can be copied from the input
* spill files to the output channel without copying data into memory. If such optimizations are
* not supported, the implementation should return {@link Optional#empty()}. By default, the
* implementation returns {@link Optional#empty()}.
* <p>
* Note that the returned {@link WritableByteChannelWrapper} itself is closed, but not the
* underlying channel that is returned by {@link WritableByteChannelWrapper#channel()}. Ensure
* that the underlying channel is cleaned up in {@link WritableByteChannelWrapper#close()},
* {@link ShuffleMapOutputWriter#commitAllPartitions()}, or
* {@link ShuffleMapOutputWriter#abort(Throwable)}.
*/
default Optional<WritableByteChannelWrapper> openChannelWrapper() throws IOException {
return Optional.empty();
}

/**
* Returns the number of bytes written either by this writer's output stream opened by
* {@link #openStream()} or the byte channel opened by {@link #openChannelWrapper()}.
* <p>
* This can be different from the number of bytes given by the caller. For example, the
* stream might compress or encrypt the bytes before persisting the data to the backing
* data store.
*/
long getNumBytesWritten();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* 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.shuffle.api;

import java.io.Closeable;
import java.nio.channels.WritableByteChannel;

import org.apache.spark.annotation.Private;

/**
* :: Private ::
*
* A thin wrapper around a {@link WritableByteChannel}.
* <p>
* This is primarily provided for the local disk shuffle implementation to provide a
* {@link java.nio.channels.FileChannel} that keeps the channel open across partition writes.
*
* @since 3.0.0
*/
@Private
public interface WritableByteChannelWrapper extends Closeable {

/**
* The underlying channel to write bytes into.
*/
WritableByteChannel channel();
}
Loading

0 comments on commit abef84a

Please sign in to comment.