From 1957e82a83f0860b5f04890984c64fc3528631ce Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 20 Mar 2019 15:14:04 -0700 Subject: [PATCH 01/32] [SPARK-25299] Introduce the new shuffle writer API (#5) (#520) Introduces the new Shuffle Writer API. Ported from https://github.com/bloomberg/apache-spark-on-k8s/pull/5. --- .../spark/api/shuffle/ShuffleDataIO.java | 31 ++++++++++++++ .../shuffle/ShuffleExecutorComponents.java | 33 +++++++++++++++ .../api/shuffle/ShuffleMapOutputWriter.java | 37 ++++++++++++++++ .../api/shuffle/ShufflePartitionWriter.java | 42 +++++++++++++++++++ .../api/shuffle/ShuffleWriteSupport.java | 37 ++++++++++++++++ 5 files changed, 180 insertions(+) create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java new file mode 100644 index 000000000000..4cb40f6dd00b --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java @@ -0,0 +1,31 @@ +/* + * 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.api.shuffle; + +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * An interface for launching Shuffle related components + * + * @since 3.0.0 + */ +@Experimental +public interface ShuffleDataIO { + ShuffleExecutorComponents executor(); +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java new file mode 100644 index 000000000000..1edf044225cc --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java @@ -0,0 +1,33 @@ +/* + * 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.api.shuffle; + +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * An interface for building shuffle support for Executors + * + * @since 3.0.0 + */ +@Experimental +public interface ShuffleExecutorComponents { + void intitializeExecutor(String appId, String execId); + + ShuffleWriteSupport writes(); +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java new file mode 100644 index 000000000000..5119e34803a8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java @@ -0,0 +1,37 @@ +/* + * 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.api.shuffle; + +import java.io.IOException; + +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * An interface for creating and managing shuffle partition writers + * + * @since 3.0.0 + */ +@Experimental +public interface ShuffleMapOutputWriter { + ShufflePartitionWriter getNextPartitionWriter() throws IOException; + + void commitAllPartitions() throws IOException; + + void abort(Throwable error) throws IOException; +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java new file mode 100644 index 000000000000..c043a6b3a499 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java @@ -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.api.shuffle; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; + +import org.apache.http.annotation.Experimental; + +/** + * :: Experimental :: + * An interface for giving streams / channels for shuffle writes + * + * @since 3.0.0 + */ +@Experimental +public interface ShufflePartitionWriter { + OutputStream openStream() throws IOException; + + long getLength(); + + default WritableByteChannel openChannel() throws IOException { + return Channels.newChannel(openStream()); + } +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java new file mode 100644 index 000000000000..5ba5564bb46d --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java @@ -0,0 +1,37 @@ +/* + * 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.api.shuffle; + +import java.io.IOException; + +import org.apache.http.annotation.Experimental; + +/** + * :: Experimental :: + * An interface for deploying a shuffle map output writer + * + * @since 3.0.0 + */ +@Experimental +public interface ShuffleWriteSupport { + ShuffleMapOutputWriter createMapOutputWriter( + String appId, + int shuffleId, + int mapId, + int numPartitions) throws IOException; +} From 857552a116d282a0fb9e3c82f1d399f23ab09e5a Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 3 Apr 2019 16:34:46 -0700 Subject: [PATCH 02/32] [SPARK-25299] Local shuffle implementation of the shuffle writer API (#524) Implements the shuffle writer API by writing shuffle files to local disk and using the index block resolver to commit data and write index files. The logic in `BypassMergeSortShuffleWriter` has been refactored to use the base implementation of the plugin instead. APIs have been slightly renamed to clarify semantics after considering nuances in how these are to be implemented by other developers. Follow-up commits are to come for `SortShuffleWriter` and `UnsafeShuffleWriter`. Ported from https://github.com/bloomberg/apache-spark-on-k8s/pull/6, credits to @ifilonenko. --- .../shuffle/ShuffleExecutorComponents.java | 2 +- .../api/shuffle/ShufflePartitionWriter.java | 44 +++- .../api/shuffle/ShuffleWriteSupport.java | 1 - .../sort/BypassMergeSortShuffleWriter.java | 146 ++++++----- .../shuffle/sort/io/DefaultShuffleDataIO.java | 36 +++ .../io/DefaultShuffleExecutorComponents.java | 51 ++++ .../io/DefaultShuffleMapOutputWriter.java | 243 ++++++++++++++++++ .../sort/io/DefaultShuffleWriteSupport.java | 47 ++++ .../spark/internal/config/package.scala | 7 + .../shuffle/sort/SortShuffleManager.scala | 21 +- .../scala/org/apache/spark/util/Utils.scala | 30 ++- .../scala/org/apache/spark/ShuffleSuite.scala | 14 +- .../BypassMergeSortShuffleWriterSuite.scala | 68 ++++- .../DefaultShuffleMapOutputWriterSuite.scala | 216 ++++++++++++++++ 14 files changed, 833 insertions(+), 93 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java create mode 100644 core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java index 1edf044225cc..4fc20bad9938 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java @@ -27,7 +27,7 @@ */ @Experimental public interface ShuffleExecutorComponents { - void intitializeExecutor(String appId, String execId); + void initializeExecutor(String appId, String execId); ShuffleWriteSupport writes(); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java index c043a6b3a499..6a53803e5d11 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java @@ -17,6 +17,7 @@ package org.apache.spark.api.shuffle; +import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; import java.nio.channels.Channels; @@ -26,17 +27,48 @@ /** * :: Experimental :: - * An interface for giving streams / channels for shuffle writes + * An interface for giving streams / channels for shuffle writes. * * @since 3.0.0 */ @Experimental -public interface ShufflePartitionWriter { - OutputStream openStream() throws IOException; +public interface ShufflePartitionWriter extends Closeable { - long getLength(); + /** + * Returns an underlying {@link OutputStream} that can write bytes to the underlying data store. + *

+ * Note that this stream itself is not closed by the caller; close the stream in the + * implementation of this interface's {@link #close()}. + */ + OutputStream toStream() throws IOException; - default WritableByteChannel openChannel() throws IOException { - return Channels.newChannel(openStream()); + /** + * Returns an underlying {@link WritableByteChannel} that can write bytes to the underlying data + * store. + *

+ * Note that this channel itself is not closed by the caller; close the channel in the + * implementation of this interface's {@link #close()}. + */ + default WritableByteChannel toChannel() throws IOException { + return Channels.newChannel(toStream()); } + + /** + * Get the number of bytes written by this writer's stream returned by {@link #toStream()} or + * the channel returned by {@link #toChannel()}. + */ + long getNumBytesWritten(); + + /** + * Close all resources created by this ShufflePartitionWriter, via calls to {@link #toStream()} + * or {@link #toChannel()}. + *

+ * This must always close any stream returned by {@link #toStream()}. + *

+ * Note that the default version of {@link #toChannel()} returns a {@link WritableByteChannel} + * that does not itself need to be closed up front; only the underlying output stream given by + * {@link #toStream()} must be closed. + */ + @Override + void close() throws IOException; } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java index 5ba5564bb46d..6c69d5db9fd0 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java @@ -30,7 +30,6 @@ @Experimental public interface ShuffleWriteSupport { ShuffleMapOutputWriter createMapOutputWriter( - String appId, int shuffleId, int mapId, int numPartitions) throws IOException; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 32b446785a9f..aef133fe7d46 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -19,8 +19,10 @@ import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; +import java.io.OutputStream; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; import javax.annotation.Nullable; import scala.None$; @@ -34,6 +36,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; +import org.apache.spark.api.shuffle.ShufflePartitionWriter; +import org.apache.spark.api.shuffle.ShuffleWriteSupport; import org.apache.spark.internal.config.package$; import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; @@ -82,6 +87,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int shuffleId; private final int mapId; private final Serializer serializer; + private final ShuffleWriteSupport shuffleWriteSupport; private final IndexShuffleBlockResolver shuffleBlockResolver; /** Array of file writers, one for each partition */ @@ -103,7 +109,8 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleHandle handle, int mapId, SparkConf conf, - ShuffleWriteMetricsReporter writeMetrics) { + ShuffleWriteMetricsReporter writeMetrics, + ShuffleWriteSupport shuffleWriteSupport) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); @@ -116,57 +123,61 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; + this.shuffleWriteSupport = shuffleWriteSupport; } @Override public void write(Iterator> records) throws IOException { assert (partitionWriters == null); - if (!records.hasNext()) { - partitionLengths = new long[numPartitions]; - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, null); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); - return; - } - final SerializerInstance serInstance = serializer.newInstance(); - final long openStartTime = System.nanoTime(); - partitionWriters = new DiskBlockObjectWriter[numPartitions]; - partitionWriterSegments = new FileSegment[numPartitions]; - for (int i = 0; i < numPartitions; i++) { - final Tuple2 tempShuffleBlockIdPlusFile = - blockManager.diskBlockManager().createTempShuffleBlock(); - final File file = tempShuffleBlockIdPlusFile._2(); - final BlockId blockId = tempShuffleBlockIdPlusFile._1(); - partitionWriters[i] = - blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); - } - // Creating the file to write to and creating a disk writer both involve interacting with - // the disk, and can take a long time in aggregate when we open many files, so should be - // included in the shuffle write time. - writeMetrics.incWriteTime(System.nanoTime() - openStartTime); - - while (records.hasNext()) { - final Product2 record = records.next(); - final K key = record._1(); - partitionWriters[partitioner.getPartition(key)].write(key, record._2()); - } + ShuffleMapOutputWriter mapOutputWriter = shuffleWriteSupport + .createMapOutputWriter(shuffleId, mapId, numPartitions); + try { + if (!records.hasNext()) { + partitionLengths = new long[numPartitions]; + mapOutputWriter.commitAllPartitions(); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + return; + } + final SerializerInstance serInstance = serializer.newInstance(); + final long openStartTime = System.nanoTime(); + partitionWriters = new DiskBlockObjectWriter[numPartitions]; + partitionWriterSegments = new FileSegment[numPartitions]; + for (int i = 0; i < numPartitions; i++) { + final Tuple2 tempShuffleBlockIdPlusFile = + blockManager.diskBlockManager().createTempShuffleBlock(); + final File file = tempShuffleBlockIdPlusFile._2(); + final BlockId blockId = tempShuffleBlockIdPlusFile._1(); + partitionWriters[i] = + blockManager.getDiskWriter(blockId, file, serInstance, fileBufferSize, writeMetrics); + } + // Creating the file to write to and creating a disk writer both involve interacting with + // the disk, and can take a long time in aggregate when we open many files, so should be + // included in the shuffle write time. + writeMetrics.incWriteTime(System.nanoTime() - openStartTime); - for (int i = 0; i < numPartitions; i++) { - try (DiskBlockObjectWriter writer = partitionWriters[i]) { - partitionWriterSegments[i] = writer.commitAndGet(); + while (records.hasNext()) { + final Product2 record = records.next(); + final K key = record._1(); + partitionWriters[partitioner.getPartition(key)].write(key, record._2()); } - } - File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); - File tmp = Utils.tempFileWith(output); - try { - partitionLengths = writePartitionedFile(tmp); - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); - } finally { - if (tmp.exists() && !tmp.delete()) { - logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + for (int i = 0; i < numPartitions; i++) { + try (DiskBlockObjectWriter writer = partitionWriters[i]) { + partitionWriterSegments[i] = writer.commitAndGet(); + } } + + partitionLengths = writePartitionedData(mapOutputWriter); + mapOutputWriter.commitAllPartitions(); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + } catch (Exception e) { + try { + mapOutputWriter.abort(e); + } catch (Exception e2) { + logger.error("Failed to abort the writer after failing to write map output.", e2); + } + throw e; } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting @@ -179,37 +190,54 @@ long[] getPartitionLengths() { * * @return array of lengths, in bytes, of each partition of the file (used by map output tracker). */ - private long[] writePartitionedFile(File outputFile) throws IOException { + private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) throws IOException { // Track location of the partition starts in the output file final long[] lengths = new long[numPartitions]; if (partitionWriters == null) { // We were passed an empty iterator return lengths; } - - final FileOutputStream out = new FileOutputStream(outputFile, true); final long writeStartTime = System.nanoTime(); - boolean threwException = true; try { for (int i = 0; i < numPartitions; i++) { final File file = partitionWriterSegments[i].file(); - if (file.exists()) { - final FileInputStream in = new FileInputStream(file); - boolean copyThrewException = true; - try { - lengths[i] = Utils.copyStream(in, out, false, transferToEnabled); + boolean copyThrewException = true; + ShufflePartitionWriter writer = null; + try { + writer = mapOutputWriter.getNextPartitionWriter(); + if (!file.exists()) { copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); - } - if (!file.delete()) { - logger.error("Unable to delete file for partition {}", i); + } else { + if (transferToEnabled) { + WritableByteChannel outputChannel = writer.toChannel(); + FileInputStream in = new FileInputStream(file); + try (FileChannel inputChannel = in.getChannel()) { + Utils.copyFileStreamNIO(inputChannel, outputChannel, 0, inputChannel.size()); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + } else { + OutputStream tempOutputStream = writer.toStream(); + FileInputStream in = new FileInputStream(file); + try { + Utils.copyStream(in, tempOutputStream, false, false); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + } + if (!file.delete()) { + logger.error("Unable to delete file for partition {}", i); + } } + } finally { + Closeables.close(writer, copyThrewException); } + + lengths[i] = writer.getNumBytesWritten(); } - threwException = false; } finally { - Closeables.close(out, threwException); writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); } partitionWriters = null; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java new file mode 100644 index 000000000000..906600c0f15f --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java @@ -0,0 +1,36 @@ +/* + * 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.sort.io; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.shuffle.ShuffleExecutorComponents; +import org.apache.spark.api.shuffle.ShuffleDataIO; + +public class DefaultShuffleDataIO implements ShuffleDataIO { + + private final SparkConf sparkConf; + + public DefaultShuffleDataIO(SparkConf sparkConf) { + this.sparkConf = sparkConf; + } + + @Override + public ShuffleExecutorComponents executor() { + return new DefaultShuffleExecutorComponents(sparkConf); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java new file mode 100644 index 000000000000..76e87a674025 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java @@ -0,0 +1,51 @@ +/* + * 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.sort.io; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkEnv; +import org.apache.spark.api.shuffle.ShuffleExecutorComponents; +import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.storage.BlockManager; + +public class DefaultShuffleExecutorComponents implements ShuffleExecutorComponents { + + private final SparkConf sparkConf; + private BlockManager blockManager; + private IndexShuffleBlockResolver blockResolver; + + public DefaultShuffleExecutorComponents(SparkConf sparkConf) { + this.sparkConf = sparkConf; + } + + @Override + public void initializeExecutor(String appId, String execId) { + blockManager = SparkEnv.get().blockManager(); + blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager); + } + + @Override + public ShuffleWriteSupport writes() { + if (blockResolver == null) { + throw new IllegalStateException( + "Executor components must be initialized before getting writers."); + } + return new DefaultShuffleWriteSupport(sparkConf, blockResolver); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java new file mode 100644 index 000000000000..0f7e5ed66bb7 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -0,0 +1,243 @@ +/* + * 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.sort.io; + +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.channels.FileChannel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; +import org.apache.spark.api.shuffle.ShufflePartitionWriter; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; +import org.apache.spark.internal.config.package$; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.storage.TimeTrackingOutputStream; +import org.apache.spark.util.Utils; + +public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { + + private static final Logger log = + LoggerFactory.getLogger(DefaultShuffleMapOutputWriter.class); + + private final int shuffleId; + private final int mapId; + private final ShuffleWriteMetricsReporter metrics; + private final IndexShuffleBlockResolver blockResolver; + private final long[] partitionLengths; + private final int bufferSize; + private int currPartitionId = 0; + private long currChannelPosition; + + private final File outputFile; + private File outputTempFile; + private FileOutputStream outputFileStream; + private FileChannel outputFileChannel; + private TimeTrackingOutputStream ts; + private BufferedOutputStream outputBufferedFileStream; + + public DefaultShuffleMapOutputWriter( + int shuffleId, + int mapId, + int numPartitions, + ShuffleWriteMetricsReporter metrics, + IndexShuffleBlockResolver blockResolver, + SparkConf sparkConf) { + this.shuffleId = shuffleId; + this.mapId = mapId; + this.metrics = metrics; + this.blockResolver = blockResolver; + this.bufferSize = + (int) (long) sparkConf.get( + package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; + this.partitionLengths = new long[numPartitions]; + this.outputFile = blockResolver.getDataFile(shuffleId, mapId); + this.outputTempFile = null; + } + + @Override + public ShufflePartitionWriter getNextPartitionWriter() throws IOException { + if (outputTempFile == null) { + outputTempFile = Utils.tempFileWith(outputFile); + } + if (outputFileChannel != null) { + currChannelPosition = outputFileChannel.position(); + } else { + currChannelPosition = 0L; + } + return new DefaultShufflePartitionWriter(currPartitionId++); + } + + @Override + public void commitAllPartitions() throws IOException { + cleanUp(); + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, outputTempFile); + } + + @Override + public void abort(Throwable error) { + try { + cleanUp(); + } catch (Exception e) { + log.error("Unable to close appropriate underlying file stream", e); + } + if (outputTempFile != null && outputTempFile.exists() && !outputTempFile.delete()) { + log.warn("Failed to delete temporary shuffle file at {}", outputTempFile.getAbsolutePath()); + } + } + + private void cleanUp() throws IOException { + if (outputBufferedFileStream != null) { + outputBufferedFileStream.close(); + } + + if (outputFileChannel != null) { + outputFileChannel.close(); + } + + if (outputFileStream != null) { + outputFileStream.close(); + } + } + + private void initStream() throws IOException { + if (outputFileStream == null) { + outputFileStream = new FileOutputStream(outputTempFile, true); + ts = new TimeTrackingOutputStream(metrics, outputFileStream); + } + if (outputBufferedFileStream == null) { + outputBufferedFileStream = new BufferedOutputStream(ts, bufferSize); + } + } + + private void initChannel() throws IOException { + if (outputFileStream == null) { + outputFileStream = new FileOutputStream(outputTempFile, true); + } + if (outputFileChannel == null) { + outputFileChannel = outputFileStream.getChannel(); + } + } + + private class DefaultShufflePartitionWriter implements ShufflePartitionWriter { + + private final int partitionId; + private PartitionWriterStream stream = null; + + private DefaultShufflePartitionWriter(int partitionId) { + this.partitionId = partitionId; + } + + @Override + public OutputStream toStream() throws IOException { + if (outputFileChannel != null) { + throw new IllegalStateException("Requested an output channel for a previous write but" + + " now an output stream has been requested. Should not be using both channels" + + " and streams to write."); + } + initStream(); + stream = new PartitionWriterStream(); + return stream; + } + + @Override + public FileChannel toChannel() throws IOException { + if (stream != null) { + throw new IllegalStateException("Requested an output stream for a previous write but" + + " now an output channel has been requested. Should not be using both channels" + + " and streams to write."); + } + initChannel(); + return outputFileChannel; + } + + @Override + public long getNumBytesWritten() { + if (outputFileChannel != null && stream == null) { + try { + long newPosition = outputFileChannel.position(); + return newPosition - currChannelPosition; + } catch (Exception e) { + log.error("The partition which failed is: {}", partitionId, e); + throw new IllegalStateException("Failed to calculate position of file channel", e); + } + } else if (stream != null) { + return stream.getCount(); + } else { + // Assume an empty partition if stream and channel are never created + return 0; + } + } + + @Override + public void close() throws IOException { + if (stream != null) { + stream.close(); + } + partitionLengths[partitionId] = getNumBytesWritten(); + } + } + + private class PartitionWriterStream extends OutputStream { + private int count = 0; + private boolean isClosed = false; + + public int getCount() { + return count; + } + + @Override + public void write(int b) throws IOException { + verifyNotClosed(); + outputBufferedFileStream.write(b); + count++; + } + + @Override + public void write(byte[] buf, int pos, int length) throws IOException { + verifyNotClosed(); + outputBufferedFileStream.write(buf, pos, length); + count += length; + } + + @Override + public void close() throws IOException { + flush(); + isClosed = true; + } + + @Override + public void flush() throws IOException { + if (!isClosed) { + outputBufferedFileStream.flush(); + } + } + + private void verifyNotClosed() { + if (isClosed) { + throw new IllegalStateException("Attempting to write to a closed block output stream."); + } + } + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java new file mode 100644 index 000000000000..f8fadd0ecfa6 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java @@ -0,0 +1,47 @@ +/* + * 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.sort.io; + +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; +import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; + +public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { + + private final SparkConf sparkConf; + private final IndexShuffleBlockResolver blockResolver; + + public DefaultShuffleWriteSupport( + SparkConf sparkConf, + IndexShuffleBlockResolver blockResolver) { + this.sparkConf = sparkConf; + this.blockResolver = blockResolver; + } + + @Override + public ShuffleMapOutputWriter createMapOutputWriter( + int shuffleId, + int mapId, + int numPartitions) { + return new DefaultShuffleMapOutputWriter( + shuffleId, mapId, numPartitions, + TaskContext.get().taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 7c332fdb8572..3a5d8bb3f838 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -24,6 +24,7 @@ import org.apache.spark.metrics.GarbageCollectionMetrics import org.apache.spark.network.shuffle.Constants import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} +import org.apache.spark.shuffle.sort.io.DefaultShuffleDataIO import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -794,6 +795,12 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val SHUFFLE_IO_PLUGIN_CLASS = + ConfigBuilder("spark.shuffle.io.plugin.class") + .doc("Name of the class to use for shuffle IO.") + .stringConf + .createWithDefault(classOf[DefaultShuffleDataIO].getName) + private[spark] val SHUFFLE_FILE_BUFFER_SIZE = ConfigBuilder("spark.shuffle.file.buffer") .doc("Size of the in-memory buffer for each shuffle file output stream, in KiB unless " + diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index b59fa8e8a3cc..5da7b5cb35e6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -20,8 +20,10 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.api.shuffle.{ShuffleDataIO, ShuffleExecutorComponents} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ +import org.apache.spark.util.Utils /** * In sort-based shuffle, incoming records are sorted according to their target partition ids, then @@ -68,6 +70,8 @@ import org.apache.spark.shuffle._ */ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { + import SortShuffleManager._ + if (!conf.getBoolean("spark.shuffle.spill", true)) { logWarning( "spark.shuffle.spill was set to false, but this configuration is ignored as of Spark 1.6+." + @@ -79,6 +83,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager */ private[this] val numMapsForShuffle = new ConcurrentHashMap[Int, Int]() + private lazy val shuffleExecutorComponents = loadShuffleExecutorComponents(conf) + override val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) /** @@ -148,7 +154,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager bypassMergeSortHandle, mapId, env.conf, - metrics) + metrics, + shuffleExecutorComponents.writes()) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => new SortShuffleWriter(shuffleBlockResolver, other, mapId, context) } @@ -205,6 +212,16 @@ private[spark] object SortShuffleManager extends Logging { true } } + + private def loadShuffleExecutorComponents(conf: SparkConf): ShuffleExecutorComponents = { + val configuredPluginClasses = conf.get(config.SHUFFLE_IO_PLUGIN_CLASS) + val maybeIO = Utils.loadExtensions( + classOf[ShuffleDataIO], Seq(configuredPluginClasses), conf) + require(maybeIO.size == 1, s"Failed to load plugins of type $configuredPluginClasses") + val executorComponents = maybeIO.head.executor() + executorComponents.initializeExecutor(conf.getAppId, SparkEnv.get.executorId) + executorComponents + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 80d70a1d4850..24042db1c413 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -24,7 +24,7 @@ import java.lang.reflect.InvocationTargetException import java.math.{MathContext, RoundingMode} import java.net._ import java.nio.ByteBuffer -import java.nio.channels.{Channels, FileChannel} +import java.nio.channels.{Channels, FileChannel, WritableByteChannel} import java.nio.charset.StandardCharsets import java.nio.file.Files import java.security.SecureRandom @@ -394,10 +394,14 @@ private[spark] object Utils extends Logging { def copyFileStreamNIO( input: FileChannel, - output: FileChannel, + output: WritableByteChannel, startPosition: Long, bytesToCopy: Long): Unit = { - val initialPos = output.position() + val outputInitialState = output match { + case outputFileChannel: FileChannel => + Some((outputFileChannel.position(), outputFileChannel)) + case _ => None + } var count = 0L // In case transferTo method transferred less data than we have required. while (count < bytesToCopy) { @@ -412,15 +416,17 @@ private[spark] object Utils extends Logging { // kernel version 2.6.32, this issue can be seen in // https://bugs.openjdk.java.net/browse/JDK-7052359 // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). - val finalPos = output.position() - val expectedPos = initialPos + bytesToCopy - assert(finalPos == expectedPos, - s""" - |Current position $finalPos do not equal to expected position $expectedPos - |after transferTo, please check your kernel version to see if it is 2.6.32, - |this is a kernel bug which will lead to unexpected behavior when using transferTo. - |You can set spark.file.transferTo = false to disable this NIO feature. - """.stripMargin) + outputInitialState.foreach { case (initialPos, outputFileChannel) => + val finalPos = outputFileChannel.position() + val expectedPos = initialPos + bytesToCopy + assert(finalPos == expectedPos, + s""" + |Current position $finalPos do not equal to expected position $expectedPos + |after transferTo, please check your kernel version to see if it is 2.6.32, + |this is a kernel bug which will lead to unexpected behavior when using transferTo. + |You can set spark.file.transferTo = false to disable this NIO feature. + """.stripMargin) + } } /** diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 8b1084a8edc7..90c790cefcca 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListene import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.ShuffleWriter import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleIndexBlockId} -import org.apache.spark.util.{MutablePair, Utils} +import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -368,7 +368,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem) val writer1 = manager.getWriter[Int, Int]( shuffleHandle, 0, context1, context1.taskMetrics.shuffleWriteMetrics) - val data1 = (1 to 10).map { x => x -> x} + val data1 = (1 to 10).map { x => x -> x } // second attempt -- also successful. We'll write out different data, // just to simulate the fact that the records may get written differently @@ -383,13 +383,17 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // simultaneously, and everything is still OK def writeAndClose( - writer: ShuffleWriter[Int, Int])( + writer: ShuffleWriter[Int, Int], + taskContext: TaskContext)( iter: Iterator[(Int, Int)]): Option[MapStatus] = { + TaskContext.setTaskContext(taskContext) val files = writer.write(iter) - writer.stop(true) + val status = writer.stop(true) + TaskContext.unset + status } val interleaver = new InterleaveIterators( - data1, writeAndClose(writer1), data2, writeAndClose(writer2)) + data1, writeAndClose(writer1, context1), data2, writeAndClose(writer2, context2)) val (mapOutput1, mapOutput2) = interleaver.run() // check that we can read the map output and it has the right data diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index fc1422dfaac7..60096edb801c 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -18,22 +18,26 @@ package org.apache.spark.shuffle.sort import java.io.File -import java.util.UUID +import java.util.{Properties, UUID} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach +import scala.util.Random import org.apache.spark._ +import org.apache.spark.api.shuffle.ShuffleWriteSupport import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -48,7 +52,9 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte private var taskMetrics: TaskMetrics = _ private var tempDir: File = _ private var outputFile: File = _ + private var writeSupport: ShuffleWriteSupport = _ private val conf: SparkConf = new SparkConf(loadDefaults = false) + .set("spark.app.id", "sampleApp") private val temporaryFilesCreated: mutable.Buffer[File] = new ArrayBuffer[File]() private val blockIdToFileMap: mutable.Map[BlockId, File] = new mutable.HashMap[BlockId, File] private var shuffleHandle: BypassMergeSortShuffleHandle[Int, Int] = _ @@ -107,9 +113,27 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(diskBlockManager.getFile(any[BlockId])).thenAnswer { (invocation: InvocationOnMock) => blockIdToFileMap(invocation.getArguments.head.asInstanceOf[BlockId]) } + + val memoryManager = new TestMemoryManager(conf) + val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) + + TaskContext.setTaskContext(new TaskContextImpl( + stageId = 0, + stageAttemptNumber = 0, + partitionId = 0, + taskAttemptId = Random.nextInt(10000), + attemptNumber = 0, + taskMemoryManager = taskMemoryManager, + localProperties = new Properties, + metricsSystem = null, + taskMetrics = taskMetrics)) + + writeSupport = new DefaultShuffleWriteSupport(conf, blockResolver) } override def afterEach(): Unit = { + TaskContext.unset() try { Utils.deleteRecursively(tempDir) blockIdToFileMap.clear() @@ -126,7 +150,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte shuffleHandle, 0, // MapId conf, - taskContext.taskMetrics().shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport ) writer.write(Iterator.empty) writer.stop( /* success = */ true) @@ -142,6 +167,33 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte } test("write with some empty partitions") { + val transferConf = conf.clone.set("spark.file.transferTo", "false") + def records: Iterator[(Int, Int)] = + Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + blockResolver, + shuffleHandle, + 0, // MapId + transferConf, + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport + ) + writer.write(records) + writer.stop( /* success = */ true) + assert(temporaryFilesCreated.nonEmpty) + assert(writer.getPartitionLengths.sum === outputFile.length()) + assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files + assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted + val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics + assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) + assert(shuffleWriteMetrics.recordsWritten === records.length) + assert(taskMetrics.diskBytesSpilled === 0) + assert(taskMetrics.memoryBytesSpilled === 0) + } + + // TODO(ifilonenko): MAKE THIS PASS + test("write with some empty partitions with transferTo") { def records: Iterator[(Int, Int)] = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) val writer = new BypassMergeSortShuffleWriter[Int, Int]( @@ -150,7 +202,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte shuffleHandle, 0, // MapId conf, - taskContext.taskMetrics().shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport ) writer.write(records) writer.stop( /* success = */ true) @@ -185,7 +238,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte shuffleHandle, 0, // MapId conf, - taskContext.taskMetrics().shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport ) intercept[SparkException] { @@ -207,7 +261,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte shuffleHandle, 0, // MapId conf, - taskContext.taskMetrics().shuffleWriteMetrics + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport ) intercept[SparkException] { writer.write((0 until 100000).iterator.map(i => { @@ -221,5 +276,4 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte writer.stop( /* success = */ false) assert(temporaryFilesCreated.count(_.exists()) === 0) } - } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala new file mode 100644 index 000000000000..22d52924a7c7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala @@ -0,0 +1,216 @@ +/* + * 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.sort.io + +import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} +import java.math.BigInteger +import java.nio.ByteBuffer + +import org.mockito.Answers.RETURNS_SMART_NULLS +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} +import org.mockito.Mock +import org.mockito.Mockito.{doAnswer, doNothing, when} +import org.mockito.MockitoAnnotations +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.util.LimitedInputStream +import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.util.Utils + +class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { + + @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ + @Mock(answer = RETURNS_SMART_NULLS) private var shuffleWriteMetrics: ShuffleWriteMetrics = _ + + private val NUM_PARTITIONS = 4 + private val D_LEN = 10 + private val data: Array[Array[Int]] = (0 until NUM_PARTITIONS).map { + p => (1 to D_LEN).map(_ + p).toArray }.toArray + + private var tempFile: File = _ + private var mergedOutputFile: File = _ + private var tempDir: File = _ + private var partitionSizesInMergedFile: Array[Long] = _ + private var conf: SparkConf = _ + private var mapOutputWriter: DefaultShuffleMapOutputWriter = _ + + override def afterEach(): Unit = { + try { + Utils.deleteRecursively(tempDir) + } finally { + super.afterEach() + } + } + + override def beforeEach(): Unit = { + MockitoAnnotations.initMocks(this) + tempDir = Utils.createTempDir(null, "test") + mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir) + tempFile = File.createTempFile("tempfile", "", tempDir) + partitionSizesInMergedFile = null + conf = new SparkConf() + .set("spark.app.id", "example.spark.app") + .set("spark.shuffle.unsafe.file.output.buffer", "16k") + when(blockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) + + doNothing().when(shuffleWriteMetrics).incWriteTime(anyLong) + + doAnswer(new Answer[Void] { + def answer(invocationOnMock: InvocationOnMock): Void = { + partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + mergedOutputFile.delete + tmp.renameTo(mergedOutputFile) + } + null + } + }).when(blockResolver) + .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) + mapOutputWriter = new DefaultShuffleMapOutputWriter( + 0, 0, NUM_PARTITIONS, shuffleWriteMetrics, blockResolver, conf) + } + + private def readRecordsFromFile(fromByte: Boolean): Array[Array[Int]] = { + var startOffset = 0L + val result = new Array[Array[Int]](NUM_PARTITIONS) + (0 until NUM_PARTITIONS).foreach { p => + val partitionSize = partitionSizesInMergedFile(p).toInt + lazy val inner = new Array[Int](partitionSize) + lazy val innerBytebuffer = ByteBuffer.allocate(partitionSize) + if (partitionSize > 0) { + val in = new FileInputStream(mergedOutputFile) + in.getChannel.position(startOffset) + val lin = new LimitedInputStream(in, partitionSize) + var nonEmpty = true + var count = 0 + while (nonEmpty) { + try { + val readBit = lin.read() + if (fromByte) { + innerBytebuffer.put(readBit.toByte) + } else { + inner(count) = readBit + } + count += 1 + } catch { + case _: Exception => + nonEmpty = false + } + } + in.close() + } + if (fromByte) { + result(p) = innerBytebuffer.array().sliding(4, 4).map { b => + new BigInteger(b).intValue() + }.toArray + } else { + result(p) = inner + } + startOffset += partitionSize + } + result + } + + test("writing to an outputstream") { + (0 until NUM_PARTITIONS).foreach{ p => + val writer = mapOutputWriter.getNextPartitionWriter + val stream = writer.toStream() + data(p).foreach { i => stream.write(i)} + stream.close() + intercept[IllegalStateException] { + stream.write(p) + } + assert(writer.getNumBytesWritten() == D_LEN) + writer.close + } + mapOutputWriter.commitAllPartitions() + val partitionLengths = (0 until NUM_PARTITIONS).map { _ => D_LEN.toDouble}.toArray + assert(partitionSizesInMergedFile === partitionLengths) + assert(mergedOutputFile.length() === partitionLengths.sum) + assert(data === readRecordsFromFile(false)) + } + + test("writing to a channel") { + (0 until NUM_PARTITIONS).foreach{ p => + val writer = mapOutputWriter.getNextPartitionWriter + val channel = writer.toChannel() + val byteBuffer = ByteBuffer.allocate(D_LEN * 4) + val intBuffer = byteBuffer.asIntBuffer() + intBuffer.put(data(p)) + assert(channel.isOpen) + channel.write(byteBuffer) + // Bytes require * 4 + assert(writer.getNumBytesWritten == D_LEN * 4) + writer.close + } + mapOutputWriter.commitAllPartitions() + val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray + assert(partitionSizesInMergedFile === partitionLengths) + assert(mergedOutputFile.length() === partitionLengths.sum) + assert(data === readRecordsFromFile(true)) + } + + test("copyStreams with an outputstream") { + (0 until NUM_PARTITIONS).foreach{ p => + val writer = mapOutputWriter.getNextPartitionWriter + val stream = writer.toStream() + val byteBuffer = ByteBuffer.allocate(D_LEN * 4) + val intBuffer = byteBuffer.asIntBuffer() + intBuffer.put(data(p)) + val in = new ByteArrayInputStream(byteBuffer.array()) + Utils.copyStream(in, stream, false, false) + in.close() + stream.close() + assert(writer.getNumBytesWritten == D_LEN * 4) + writer.close + } + mapOutputWriter.commitAllPartitions() + val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray + assert(partitionSizesInMergedFile === partitionLengths) + assert(mergedOutputFile.length() === partitionLengths.sum) + assert(data === readRecordsFromFile(true)) + } + + test("copyStreamsWithNIO with a channel") { + (0 until NUM_PARTITIONS).foreach{ p => + val writer = mapOutputWriter.getNextPartitionWriter + val channel = writer.toChannel() + val byteBuffer = ByteBuffer.allocate(D_LEN * 4) + val intBuffer = byteBuffer.asIntBuffer() + intBuffer.put(data(p)) + val out = new FileOutputStream(tempFile) + out.write(byteBuffer.array()) + out.close() + val in = new FileInputStream(tempFile) + Utils.copyFileStreamNIO(in.getChannel, channel, 0, D_LEN * 4) + in.close() + assert(writer.getNumBytesWritten == D_LEN * 4) + writer.close + } + mapOutputWriter.commitAllPartitions() + val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray + assert(partitionSizesInMergedFile === partitionLengths) + assert(mergedOutputFile.length() === partitionLengths.sum) + assert(data === readRecordsFromFile(true)) + } +} From d13037fe6335dae6728012237fb139adaa94c29f Mon Sep 17 00:00:00 2001 From: mccheah Date: Tue, 16 Apr 2019 19:13:31 -0700 Subject: [PATCH 03/32] [SPARK-25299] Make UnsafeShuffleWriter use the new API (#536) Ported from https://github.com/bloomberg/apache-spark-on-k8s/pull/9. Credits to @ifilonenko! --- .../shuffle/sort/UnsafeShuffleWriter.java | 236 +++++++++--------- .../io/DefaultShuffleMapOutputWriter.java | 18 +- .../shuffle/sort/SortShuffleManager.scala | 3 +- .../sort/UnsafeShuffleWriterSuite.java | 43 +++- 4 files changed, 156 insertions(+), 144 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 9d05f03613ce..b5ca6c8a50ce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -20,6 +20,7 @@ import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; import java.util.Iterator; import scala.Option; @@ -31,18 +32,19 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; -import com.google.common.io.Files; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.*; import org.apache.spark.annotation.Private; +import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; +import org.apache.spark.api.shuffle.ShufflePartitionWriter; +import org.apache.spark.api.shuffle.ShuffleWriteSupport; import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; import org.apache.commons.io.output.CloseShieldOutputStream; -import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; @@ -53,7 +55,6 @@ import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; -import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; @@ -65,7 +66,6 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); @VisibleForTesting - static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; @@ -74,6 +74,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final SerializerInstance serializer; private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; + private final ShuffleWriteSupport shuffleWriteSupport; private final int shuffleId; private final int mapId; private final TaskContext taskContext; @@ -81,7 +82,6 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final boolean transferToEnabled; private final int initialSortBufferSize; private final int inputBufferSizeInBytes; - private final int outputBufferSizeInBytes; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -103,18 +103,6 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream */ private boolean stopping = false; - private class CloseAndFlushShieldOutputStream extends CloseShieldOutputStream { - - CloseAndFlushShieldOutputStream(OutputStream outputStream) { - super(outputStream); - } - - @Override - public void flush() { - // do nothing - } - } - public UnsafeShuffleWriter( BlockManager blockManager, IndexShuffleBlockResolver shuffleBlockResolver, @@ -123,7 +111,8 @@ public UnsafeShuffleWriter( int mapId, TaskContext taskContext, SparkConf sparkConf, - ShuffleWriteMetricsReporter writeMetrics) throws IOException { + ShuffleWriteMetricsReporter writeMetrics, + ShuffleWriteSupport shuffleWriteSupport) throws IOException { final int numPartitions = handle.dependency().partitioner().numPartitions(); if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( @@ -140,6 +129,7 @@ public UnsafeShuffleWriter( this.serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = writeMetrics; + this.shuffleWriteSupport = shuffleWriteSupport; this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); @@ -147,8 +137,6 @@ public UnsafeShuffleWriter( (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; - this.outputBufferSizeInBytes = - (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); } @@ -230,24 +218,27 @@ void closeAndWriteOutput() throws IOException { serOutputStream = null; final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; + final ShuffleMapOutputWriter mapWriter = shuffleWriteSupport + .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); final long[] partitionLengths; - final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); - final File tmp = Utils.tempFileWith(output); try { try { - partitionLengths = mergeSpills(spills, tmp); + partitionLengths = mergeSpills(spills, mapWriter); } finally { for (SpillInfo spill : spills) { - if (spill.file.exists() && ! spill.file.delete()) { + if (spill.file.exists() && !spill.file.delete()) { logger.error("Error while deleting spill file {}", spill.file.getPath()); } } } - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); - } finally { - if (tmp.exists() && !tmp.delete()) { - logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + mapWriter.commitAllPartitions(); + } catch (Exception e) { + try { + mapWriter.abort(e); + } catch (Exception innerE) { + logger.error("Failed to abort the Map Output Writer", innerE); } + throw e; } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @@ -281,7 +272,8 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { + private long[] mergeSpills(SpillInfo[] spills, + ShuffleMapOutputWriter mapWriter) throws IOException { final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = @@ -289,17 +281,24 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); + final int numPartitions = partitioner.numPartitions(); + long[] partitionLengths = new long[numPartitions]; try { if (spills.length == 0) { - new FileOutputStream(outputFile).close(); // Create an empty file - return new long[partitioner.numPartitions()]; - } else if (spills.length == 1) { - // Here, we don't need to perform any metrics updates because the bytes written to this - // output file would have already been counted as shuffle bytes written. - Files.move(spills[0].file, outputFile); - return spills[0].partitionLengths; + // The contract we are working under states that we will open a partition writer for + // each partition, regardless of number of spills + for (int i = 0; i < numPartitions; i++) { + ShufflePartitionWriter writer = null; + try { + writer = mapWriter.getNextPartitionWriter(); + } finally { + if (writer != null) { + writer.close(); + } + } + } + return partitionLengths; } else { - final long[] partitionLengths; // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, // then the final output file's size won't necessarily be equal to the sum of the spill @@ -316,14 +315,14 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti // that doesn't need to interpret the spilled bytes. if (transferToEnabled && !encryptionEnabled) { logger.debug("Using transferTo-based fast merge"); - partitionLengths = mergeSpillsWithTransferTo(spills, outputFile); + partitionLengths = mergeSpillsWithTransferTo(spills, mapWriter); } else { logger.debug("Using fileStream-based fast merge"); - partitionLengths = mergeSpillsWithFileStream(spills, outputFile, null); + partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, null); } } else { logger.debug("Using slow merge"); - partitionLengths = mergeSpillsWithFileStream(spills, outputFile, compressionCodec); + partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, compressionCodec); } // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has // in-memory records, we write out the in-memory records to a file but do not count that @@ -331,13 +330,9 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); - writeMetrics.incBytesWritten(outputFile.length()); return partitionLengths; } } catch (IOException e) { - if (outputFile.exists() && !outputFile.delete()) { - logger.error("Unable to delete output file {}", outputFile.getPath()); - } throw e; } } @@ -345,73 +340,79 @@ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOExcepti /** * Merges spill files using Java FileStreams. This code path is typically slower than * the NIO-based merge, {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[], - * File)}, and it's mostly used in cases where the IO compression codec does not support - * concatenation of compressed data, when encryption is enabled, or when users have - * explicitly disabled use of {@code transferTo} in order to work around kernel bugs. + * ShuffleMapOutputWriter)}, and it's mostly used in cases where the IO compression codec + * does not support concatenation of compressed data, when encryption is enabled, or when + * users have explicitly disabled use of {@code transferTo} in order to work around kernel bugs. * This code path might also be faster in cases where individual partition size in a spill * is small and UnsafeShuffleWriter#mergeSpillsWithTransferTo method performs many small * disk ios which is inefficient. In those case, Using large buffers for input and output * files helps reducing the number of disk ios, making the file merging faster. * * @param spills the spills to merge. - * @param outputFile the file to write the merged data to. + * @param mapWriter the map output writer to use for output. * @param compressionCodec the IO compression codec, or null if shuffle compression is disabled. * @return the partition lengths in the merged file. */ private long[] mergeSpillsWithFileStream( SpillInfo[] spills, - File outputFile, + ShuffleMapOutputWriter mapWriter, @Nullable CompressionCodec compressionCodec) throws IOException { - assert (spills.length >= 2); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final InputStream[] spillInputStreams = new InputStream[spills.length]; - final OutputStream bos = new BufferedOutputStream( - new FileOutputStream(outputFile), - outputBufferSizeInBytes); - // Use a counting output stream to avoid having to close the underlying file and ask - // the file system for its size after each partition is written. - final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos); - boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputStreams[i] = new NioBufferedFileInputStream( - spills[i].file, - inputBufferSizeInBytes); + spills[i].file, + inputBufferSizeInBytes); } for (int partition = 0; partition < numPartitions; partition++) { - final long initialFileLength = mergedFileOutputStream.getByteCount(); - // Shield the underlying output stream from close() and flush() calls, so that we can close - // the higher level streams to make sure all data is really flushed and internal state is - // cleaned. - OutputStream partitionOutput = new CloseAndFlushShieldOutputStream( - new TimeTrackingOutputStream(writeMetrics, mergedFileOutputStream)); - partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); - if (compressionCodec != null) { - partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); - } - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i], - partitionLengthInSpill, false); - try { - partitionInputStream = blockManager.serializerManager().wrapForEncryption( - partitionInputStream); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); + boolean copyThrewExecption = true; + ShufflePartitionWriter writer = null; + try { + writer = mapWriter.getNextPartitionWriter(); + OutputStream partitionOutput = null; + try { + // Shield the underlying output stream from close() calls, so that we can close the + // higher level streams to make sure all data is really flushed and internal state + // is cleaned + partitionOutput = new CloseShieldOutputStream(writer.toStream()); + partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); + if (compressionCodec != null) { + partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); + } + for (int i = 0; i < spills.length; i++) { + final long partitionLengthInSpill = spills[i].partitionLengths[partition]; + + if (partitionLengthInSpill > 0) { + InputStream partitionInputStream = null; + try { + partitionInputStream = new LimitedInputStream(spillInputStreams[i], + partitionLengthInSpill, false); + partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionInputStream); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream( + partitionInputStream); + } + ByteStreams.copy(partitionInputStream, partitionOutput); + } finally { + partitionInputStream.close(); + } } - ByteStreams.copy(partitionInputStream, partitionOutput); - } finally { - partitionInputStream.close(); + copyThrewExecption = false; } + } finally { + Closeables.close(partitionOutput, copyThrewExecption); } + } finally { + Closeables.close(writer, copyThrewExecption); } - partitionOutput.flush(); - partitionOutput.close(); - partitionLengths[partition] = (mergedFileOutputStream.getByteCount() - initialFileLength); + long numBytesWritten = writer.getNumBytesWritten(); + partitionLengths[partition] = numBytesWritten; + writeMetrics.incBytesWritten(numBytesWritten); } threwException = false; } finally { @@ -420,7 +421,6 @@ private long[] mergeSpillsWithFileStream( for (InputStream stream : spillInputStreams) { Closeables.close(stream, threwException); } - Closeables.close(mergedFileOutputStream, threwException); } return partitionLengths; } @@ -430,54 +430,49 @@ private long[] mergeSpillsWithFileStream( * This is only safe when the IO compression codec and serializer support concatenation of * serialized streams. * + * @param spills the spills to merge. + * @param mapWriter the map output writer to use for output. * @return the partition lengths in the merged file. */ - private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) throws IOException { - assert (spills.length >= 2); + private long[] mergeSpillsWithTransferTo( + SpillInfo[] spills, + ShuffleMapOutputWriter mapWriter) throws IOException { final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final FileChannel[] spillInputChannels = new FileChannel[spills.length]; final long[] spillInputChannelPositions = new long[spills.length]; - FileChannel mergedFileOutputChannel = null; boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputChannels[i] = new FileInputStream(spills[i].file).getChannel(); } - // This file needs to opened in append mode in order to work around a Linux kernel bug that - // affects transferTo; see SPARK-3948 for more details. - mergedFileOutputChannel = new FileOutputStream(outputFile, true).getChannel(); - - long bytesWrittenToMergedFile = 0; for (int partition = 0; partition < numPartitions; partition++) { - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - final FileChannel spillInputChannel = spillInputChannels[i]; - final long writeStartTime = System.nanoTime(); - Utils.copyFileStreamNIO( - spillInputChannel, - mergedFileOutputChannel, - spillInputChannelPositions[i], - partitionLengthInSpill); - spillInputChannelPositions[i] += partitionLengthInSpill; - writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); - bytesWrittenToMergedFile += partitionLengthInSpill; - partitionLengths[partition] += partitionLengthInSpill; + boolean copyThrewExecption = true; + ShufflePartitionWriter writer = null; + try { + writer = mapWriter.getNextPartitionWriter(); + WritableByteChannel channel = writer.toChannel(); + for (int i = 0; i < spills.length; i++) { + long partitionLengthInSpill = 0L; + partitionLengthInSpill += spills[i].partitionLengths[partition]; + final FileChannel spillInputChannel = spillInputChannels[i]; + final long writeStartTime = System.nanoTime(); + Utils.copyFileStreamNIO( + spillInputChannel, + channel, + spillInputChannelPositions[i], + partitionLengthInSpill); + copyThrewExecption = false; + spillInputChannelPositions[i] += partitionLengthInSpill; + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); + } + } finally { + Closeables.close(writer, copyThrewExecption); } - } - // Check the position after transferTo loop to see if it is in the right position and raise an - // exception if it is incorrect. The position will not be increased to the expected length - // after calling transferTo in kernel version 2.6.32. This issue is described at - // https://bugs.openjdk.java.net/browse/JDK-7052359 and SPARK-3948. - if (mergedFileOutputChannel.position() != bytesWrittenToMergedFile) { - throw new IOException( - "Current position " + mergedFileOutputChannel.position() + " does not equal expected " + - "position " + bytesWrittenToMergedFile + " after transferTo. Please check your kernel" + - " version to see if it is 2.6.32, as there is a kernel bug which will lead to " + - "unexpected behavior when using transferTo. You can set spark.file.transferTo=false " + - "to disable this NIO feature." - ); + long numBytes = writer.getNumBytesWritten(); + partitionLengths[partition] = numBytes; + writeMetrics.incBytesWritten(numBytes); } threwException = false; } finally { @@ -487,7 +482,6 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th assert(spillInputChannelPositions[i] == spills[i].file.length()); Closeables.close(spillInputChannels[i], threwException); } - Closeables.close(mergedFileOutputChannel, threwException); } return partitionLengths; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index 0f7e5ed66bb7..c84158e1891d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -92,7 +92,8 @@ public ShufflePartitionWriter getNextPartitionWriter() throws IOException { @Override public void commitAllPartitions() throws IOException { cleanUp(); - blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, outputTempFile); + File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); } @Override @@ -111,11 +112,9 @@ private void cleanUp() throws IOException { if (outputBufferedFileStream != null) { outputBufferedFileStream.close(); } - if (outputFileChannel != null) { outputFileChannel.close(); } - if (outputFileStream != null) { outputFileStream.close(); } @@ -191,8 +190,9 @@ public long getNumBytesWritten() { } @Override - public void close() throws IOException { + public void close() { if (stream != null) { + // Closing is a no-op. stream.close(); } partitionLengths[partitionId] = getNumBytesWritten(); @@ -222,18 +222,10 @@ public void write(byte[] buf, int pos, int length) throws IOException { } @Override - public void close() throws IOException { - flush(); + public void close() { isClosed = true; } - @Override - public void flush() throws IOException { - if (!isClosed) { - outputBufferedFileStream.flush(); - } - } - private void verifyNotClosed() { if (isClosed) { throw new IllegalStateException("Attempting to write to a closed block output stream."); diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 5da7b5cb35e6..6925a542a86c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -146,7 +146,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager mapId, context, env.conf, - metrics) + metrics, + shuffleExecutorComponents.writes()) case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new BypassMergeSortShuffleWriter( env.blockManager, diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 88125a6b93ad..d37673aa69c3 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -19,8 +19,10 @@ import java.io.*; import java.nio.ByteBuffer; +import java.nio.file.Files; import java.util.*; +import org.mockito.stubbing.Answer; import scala.Option; import scala.Product2; import scala.Tuple2; @@ -39,6 +41,7 @@ import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.io.CompressionCodec$; @@ -53,6 +56,7 @@ import org.apache.spark.security.CryptoStreamUtils; import org.apache.spark.serializer.*; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -65,6 +69,7 @@ public class UnsafeShuffleWriterSuite { + static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int NUM_PARTITITONS = 4; TestMemoryManager memoryManager; TaskMemoryManager taskMemoryManager; @@ -85,6 +90,7 @@ public class UnsafeShuffleWriterSuite { @After public void tearDown() { + TaskContext$.MODULE$.unset(); Utils.deleteRecursively(tempDir); final long leakedMemory = taskMemoryManager.cleanUpAllAllocatedMemory(); if (leakedMemory != 0) { @@ -132,14 +138,28 @@ public void setUp() throws IOException { }); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - doAnswer(invocationOnMock -> { + + Answer renameTempAnswer = invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; - mergedOutputFile.delete(); - tmp.renameTo(mergedOutputFile); + if (!mergedOutputFile.delete()) { + throw new RuntimeException("Failed to delete old merged output file."); + } + if (tmp != null) { + Files.move(tmp.toPath(), mergedOutputFile.toPath()); + } else if (!mergedOutputFile.createNewFile()) { + throw new RuntimeException("Failed to create empty merged output file."); + } return null; - }).when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + }; + + doAnswer(renameTempAnswer) + .when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); + + doAnswer(renameTempAnswer) + .when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), eq(null)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -151,6 +171,9 @@ public void setUp() throws IOException { when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(shuffleDep.serializer()).thenReturn(serializer); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); + + TaskContext$.MODULE$.setTaskContext(taskContext); } private UnsafeShuffleWriter createWriter( @@ -164,7 +187,8 @@ private UnsafeShuffleWriter createWriter( 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics() + taskContext.taskMetrics().shuffleWriteMetrics(), + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver) ); } @@ -444,10 +468,10 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() thro } private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { - memoryManager.limit(UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); + memoryManager.limit(DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); - for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { + for (int i = 0; i < DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { dataToWrite.add(new Tuple2<>(i, i)); } writer.write(dataToWrite.iterator()); @@ -525,7 +549,8 @@ public void testPeakMemoryUsed() throws Exception { 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics()); + taskContext.taskMetrics().shuffleWriteMetrics(), + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. From 8f5fb60bbf839a2b53f73207908a7982f46691d4 Mon Sep 17 00:00:00 2001 From: mccheah Date: Mon, 15 Apr 2019 11:35:03 -0700 Subject: [PATCH 04/32] [SPARK-25299] Use the shuffle writer plugin for the SortShuffleWriter. (#532) * [SPARK-25299] Use the shuffle writer plugin for the SortShuffleWriter. * Remove unused * Handle empty partitions properly. * Adjust formatting * Don't close streams twice. Because compressed output streams don't like it. * Clarify comment --- .../shuffle/sort/SortShuffleManager.scala | 3 +- .../shuffle/sort/SortShuffleWriter.scala | 23 ++-- .../spark/storage/DiskBlockObjectWriter.scala | 4 +- .../util/collection/ExternalSorter.scala | 130 +++++++++++++++++- .../spark/util/collection/PairsWriter.scala | 23 ++++ .../ShufflePartitionPairsWriter.scala | 105 ++++++++++++++ .../WritablePartitionedPairCollection.scala | 4 +- 7 files changed, 266 insertions(+), 26 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 6925a542a86c..849050556c56 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -158,7 +158,8 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager metrics, shuffleExecutorComponents.writes()) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => - new SortShuffleWriter(shuffleBlockResolver, other, mapId, context) + new SortShuffleWriter( + shuffleBlockResolver, other, mapId, context, shuffleExecutorComponents.writes()) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 16058de8bf3f..62316f384b64 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,18 +18,18 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ +import org.apache.spark.api.shuffle.ShuffleWriteSupport import org.apache.spark.internal.{config, 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.util.Utils import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], mapId: Int, - context: TaskContext) + context: TaskContext, + writeSupport: ShuffleWriteSupport) extends ShuffleWriter[K, V] with Logging { private val dep = handle.dependency @@ -64,18 +64,11 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) - val tmp = Utils.tempFileWith(output) - try { - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, tmp) - shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) - } finally { - if (tmp.exists() && !tmp.delete()) { - logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") - } - } + val mapOutputWriter = writeSupport.createMapOutputWriter( + dep.shuffleId, mapId, dep.partitioner.numPartitions) + val partitionLengths = sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) + mapOutputWriter.commitAllPartitions() + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index 17390f9c60e7..f9f4e3594e4f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -24,6 +24,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.util.Utils +import org.apache.spark.util.collection.PairsWriter /** * A class for writing JVM objects directly to a file on disk. This class allows data to be appended @@ -46,7 +47,8 @@ private[spark] class DiskBlockObjectWriter( writeMetrics: ShuffleWriteMetricsReporter, val blockId: BlockId = null) extends OutputStream - with Logging { + with Logging + with PairsWriter { /** * Guards against close calls, e.g. from a wrapping stream. diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 3f3b7d20eb16..445db7cac60c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -26,10 +26,11 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams import org.apache.spark._ +import org.apache.spark.api.shuffle.{ShuffleMapOutputWriter, ShufflePartitionWriter} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer._ -import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} +import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -670,11 +671,9 @@ private[spark] class ExternalSorter[K, V, C]( } /** - * Write all the data added into this ExternalSorter into a file in the disk store. This is - * called by the SortShuffleWriter. - * - * @param blockId block ID to write to. The index file will be blockId.name + ".index". - * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) + * TODO remove this, as this is only used by UnsafeRowSerializerSuite in the SQL project. + * We should figure out an alternative way to test that so that we can remove this otherwise + * unused code path. */ def writePartitionedFile( blockId: BlockId, @@ -718,6 +717,123 @@ private[spark] class ExternalSorter[K, V, C]( lengths } + private def writeEmptyPartition(mapOutputWriter: ShuffleMapOutputWriter): Unit = { + var partitionWriter: ShufflePartitionWriter = null + try { + partitionWriter = mapOutputWriter.getNextPartitionWriter + } finally { + if (partitionWriter != null) { + partitionWriter.close() + } + } + } + + /** + * Write all the data added into this ExternalSorter into a map output writer that pushes bytes + * to some arbitrary backing store. This is called by the SortShuffleWriter. + * + * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) + */ + def writePartitionedMapOutput( + shuffleId: Int, mapId: Int, mapOutputWriter: ShuffleMapOutputWriter): Array[Long] = { + // Track location of each range in the map output + val lengths = new Array[Long](numPartitions) + var nextPartitionId = 0 + if (spills.isEmpty) { + // Case where we only have in-memory data + val collection = if (aggregator.isDefined) map else buffer + val it = collection.destructiveSortedWritablePartitionedIterator(comparator) + while (it.hasNext()) { + val partitionId = it.nextPartition() + // The contract for the plugin is that we will ask for a writer for every partition + // even if it's empty. However, the external sorter will return non-contiguous + // partition ids. So this loop "backfills" the empty partitions that form the gaps. + + // The algorithm as a whole is correct because the partition ids are returned by the + // iterator in ascending order. + for (emptyPartition <- nextPartitionId until partitionId) { + writeEmptyPartition(mapOutputWriter) + } + var partitionWriter: ShufflePartitionWriter = null + var partitionPairsWriter: ShufflePartitionPairsWriter = null + try { + partitionWriter = mapOutputWriter.getNextPartitionWriter + val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) + partitionPairsWriter = new ShufflePartitionPairsWriter( + partitionWriter, + serializerManager, + serInstance, + blockId, + context.taskMetrics().shuffleWriteMetrics) + while (it.hasNext && it.nextPartition() == partitionId) { + it.writeNext(partitionPairsWriter) + } + } finally { + if (partitionPairsWriter != null) { + partitionPairsWriter.close() + } + if (partitionWriter != null) { + partitionWriter.close() + } + } + if (partitionWriter != null) { + lengths(partitionId) = partitionWriter.getNumBytesWritten + } + nextPartitionId = partitionId + 1 + } + } else { + // We must perform merge-sort; get an iterator by partition and write everything directly. + for ((id, elements) <- this.partitionedIterator) { + // The contract for the plugin is that we will ask for a writer for every partition + // even if it's empty. However, the external sorter will return non-contiguous + // partition ids. So this loop "backfills" the empty partitions that form the gaps. + + // The algorithm as a whole is correct because the partition ids are returned by the + // iterator in ascending order. + for (emptyPartition <- nextPartitionId until id) { + writeEmptyPartition(mapOutputWriter) + } + val blockId = ShuffleBlockId(shuffleId, mapId, id) + var partitionWriter: ShufflePartitionWriter = null + var partitionPairsWriter: ShufflePartitionPairsWriter = null + try { + partitionWriter = mapOutputWriter.getNextPartitionWriter + partitionPairsWriter = new ShufflePartitionPairsWriter( + partitionWriter, + serializerManager, + serInstance, + blockId, + context.taskMetrics().shuffleWriteMetrics) + if (elements.hasNext) { + for (elem <- elements) { + partitionPairsWriter.write(elem._1, elem._2) + } + } + } finally { + if (partitionPairsWriter!= null) { + partitionPairsWriter.close() + } + } + if (partitionWriter != null) { + lengths(id) = partitionWriter.getNumBytesWritten + } + nextPartitionId = id + 1 + } + } + + // The iterator may have stopped short of opening a writer for every partition. So fill in the + // remaining empty partitions. + for (emptyPartition <- nextPartitionId until numPartitions) { + writeEmptyPartition(mapOutputWriter) + } + + context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) + + lengths + } + def stop(): Unit = { spills.foreach(s => s.file.delete()) spills.clear() @@ -781,7 +897,7 @@ private[spark] class ExternalSorter[K, V, C]( val inMemoryIterator = new WritablePartitionedIterator { private[this] var cur = if (upstream.hasNext) upstream.next() else null - def writeNext(writer: DiskBlockObjectWriter): Unit = { + def writeNext(writer: PairsWriter): Unit = { writer.write(cur._1._2, cur._2) cur = if (upstream.hasNext) upstream.next() else null } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala new file mode 100644 index 000000000000..9d7c209f242e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala @@ -0,0 +1,23 @@ +/* + * 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.util.collection + +private[spark] trait PairsWriter { + + def write(key: Any, value: Any): Unit +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala new file mode 100644 index 000000000000..6f19a2323efd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala @@ -0,0 +1,105 @@ +/* + * 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.util.collection + +import java.io.{Closeable, FilterOutputStream, OutputStream} + +import org.apache.spark.api.shuffle.ShufflePartitionWriter +import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter +import org.apache.spark.storage.BlockId + +/** + * A key-value writer inspired by {@link DiskBlockObjectWriter} that pushes the bytes to an + * arbitrary partition writer instead of writing to local disk through the block manager. + */ +private[spark] class ShufflePartitionPairsWriter( + partitionWriter: ShufflePartitionWriter, + serializerManager: SerializerManager, + serializerInstance: SerializerInstance, + blockId: BlockId, + writeMetrics: ShuffleWriteMetricsReporter) + extends PairsWriter with Closeable { + + private var isOpen = false + private var partitionStream: OutputStream = _ + private var wrappedStream: OutputStream = _ + private var objOut: SerializationStream = _ + private var numRecordsWritten = 0 + private var curNumBytesWritten = 0L + + override def write(key: Any, value: Any): Unit = { + if (!isOpen) { + open() + isOpen = true + } + objOut.writeKey(key) + objOut.writeValue(value) + writeMetrics.incRecordsWritten(1) + } + + private def open(): Unit = { + // The contract is that the partition writer is expected to close its own streams, but + // the compressor will only flush the stream when it is specifically closed. So we want to + // close objOut to flush the compressed bytes to the partition writer stream, but we don't want + // to close the partition output stream in the process. + partitionStream = new CloseShieldOutputStream(partitionWriter.toStream) + wrappedStream = serializerManager.wrapStream(blockId, partitionStream) + objOut = serializerInstance.serializeStream(wrappedStream) + } + + override def close(): Unit = { + if (isOpen) { + // Closing objOut should propagate close to all inner layers + // We can't close wrappedStream explicitly because closing objOut and closing wrappedStream + // causes problems when closing compressed output streams twice. + objOut.close() + objOut = null + wrappedStream = null + partitionStream = null + partitionWriter.close() + isOpen = false + updateBytesWritten() + } + } + + /** + * Notify the writer that a record worth of bytes has been written with OutputStream#write. + */ + private def recordWritten(): Unit = { + numRecordsWritten += 1 + writeMetrics.incRecordsWritten(1) + + if (numRecordsWritten % 16384 == 0) { + updateBytesWritten() + } + } + + private def updateBytesWritten(): Unit = { + val numBytesWritten = partitionWriter.getNumBytesWritten + val bytesWrittenDiff = numBytesWritten - curNumBytesWritten + writeMetrics.incBytesWritten(bytesWrittenDiff) + curNumBytesWritten = numBytesWritten + } + + private class CloseShieldOutputStream(delegate: OutputStream) + extends FilterOutputStream(delegate) { + + override def close(): Unit = flush() + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala index dd7f68fd038d..da8d58d05b6b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -52,7 +52,7 @@ private[spark] trait WritablePartitionedPairCollection[K, V] { new WritablePartitionedIterator { private[this] var cur = if (it.hasNext) it.next() else null - def writeNext(writer: DiskBlockObjectWriter): Unit = { + def writeNext(writer: PairsWriter): Unit = { writer.write(cur._1._2, cur._2) cur = if (it.hasNext) it.next() else null } @@ -89,7 +89,7 @@ private[spark] object WritablePartitionedPairCollection { * has an associated partition. */ private[spark] trait WritablePartitionedIterator { - def writeNext(writer: DiskBlockObjectWriter): Unit + def writeNext(writer: PairsWriter): Unit def hasNext(): Boolean From e17c7ea6da6be494892f26ce687b4eb8d666ea76 Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 19 Apr 2019 13:17:02 -0700 Subject: [PATCH 05/32] [SPARK-25299] Shuffle locations api (#517) Implements the shuffle locations API as part of SPARK-25299. This adds an additional field to all `MapStatus` objects: a `MapShuffleLocations` that indicates where a task's map output is stored. This module is optional and implementations of the pluggable shuffle writers and readers can ignore it accordingly. This API is designed with the use case in mind of future plugin implementations desiring to have the driver store metadata about where shuffle blocks are stored. There are a few caveats to this design: - We originally wanted to remove the `BlockManagerId` from `MapStatus` entirely and replace it with this object. However, doing this proves to be very difficult, as many places use the block manager ID for other kinds of shuffle data bookkeeping. As a result, we concede to storing the block manager ID redundantly here. However, the overhead should be minimal: because we cache block manager ids and default map shuffle locations, the two fields in `MapStatus` should point to the same object on the heap. Thus we add `O(M)` storage overhead on the driver, where for each map status we're storing an additional pointer to the same on-heap object. We will run benchmarks against the TPC-DS workload to see if there are significant performance repercussions for this implementation. - `KryoSerializer` expects `CompressedMapStatus` and `HighlyCompressedMapStatus` to be serialized via reflection, so originally all fields of these classes needed to be registered with Kryo. However, the `MapShuffleLocations` is now pluggable. We think however that previously Kryo was defaulting to Java serialization anyways, so we now just explicitly tell Kryo to use `ExternalizableSerializer` to deal with these objects. There's a small hack in the serialization protocol that attempts to avoid serializing the same `BlockManagerId` twice in the case that the map shuffle locations is a `DefaultMapShuffleLocations`. --- .../api/shuffle/MapShuffleLocations.java | 39 ++++++ .../spark/api/shuffle/ShuffleLocation.java | 25 ++++ .../api/shuffle/ShuffleMapOutputWriter.java | 3 +- .../sort/BypassMergeSortShuffleWriter.java | 20 ++- .../sort/DefaultMapShuffleLocations.java | 76 +++++++++++ .../shuffle/sort/UnsafeShuffleWriter.java | 10 +- .../io/DefaultShuffleExecutorComponents.java | 2 +- .../io/DefaultShuffleMapOutputWriter.java | 10 +- .../sort/io/DefaultShuffleWriteSupport.java | 8 +- .../org/apache/spark/MapOutputTracker.scala | 29 ++-- .../apache/spark/scheduler/MapStatus.scala | 125 +++++++++++++++--- .../spark/serializer/KryoSerializer.scala | 6 +- .../shuffle/BlockStoreShuffleReader.scala | 12 +- .../shuffle/sort/SortShuffleWriter.scala | 7 +- .../apache/spark/storage/BlockManagerId.scala | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 7 +- .../apache/spark/MapOutputTrackerSuite.scala | 39 +++--- .../scala/org/apache/spark/ShuffleSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 81 +++++++----- .../spark/scheduler/MapStatusSuite.scala | 28 ++-- .../scheduler/SchedulerIntegrationSuite.scala | 3 +- .../serializer/KryoSerializerSuite.scala | 7 +- .../BlockStoreShuffleReaderSuite.scala | 8 +- .../BypassMergeSortShuffleWriterSuite.scala | 6 +- .../DefaultShuffleMapOutputWriterSuite.scala | 9 +- 25 files changed, 450 insertions(+), 120 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java new file mode 100644 index 000000000000..b0aed4d08d38 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java @@ -0,0 +1,39 @@ +/* + * 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.api.shuffle; + +import org.apache.spark.annotation.Experimental; + +import java.io.Serializable; + +/** + * Represents metadata about where shuffle blocks were written in a single map task. + *

+ * This is optionally returned by shuffle writers. The inner shuffle locations may + * be accessed by shuffle readers. Shuffle locations are only necessary when the + * location of shuffle blocks needs to be managed by the driver; shuffle plugins + * may choose to use an external database or other metadata management systems to + * track the locations of shuffle blocks instead. + */ +@Experimental +public interface MapShuffleLocations extends Serializable { + + /** + * Get the location for a given shuffle block written by this map task. + */ + ShuffleLocation getLocationForBlock(int reduceId); +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java new file mode 100644 index 000000000000..87eb497098e0 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java @@ -0,0 +1,25 @@ +/* + * 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.api.shuffle; + +/** + * Marker interface representing a location of a shuffle block. Implementations of shuffle readers + * and writers are expected to cast this down to an implementation-specific representation. + */ +public interface ShuffleLocation { +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java index 5119e34803a8..181701175d35 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.spark.annotation.Experimental; +import org.apache.spark.api.java.Optional; /** * :: Experimental :: @@ -31,7 +32,7 @@ public interface ShuffleMapOutputWriter { ShufflePartitionWriter getNextPartitionWriter() throws IOException; - void commitAllPartitions() throws IOException; + Optional commitAllPartitions() throws IOException; void abort(Throwable error) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index aef133fe7d46..434286175e41 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -25,6 +25,8 @@ import java.nio.channels.WritableByteChannel; import javax.annotation.Nullable; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.shuffle.MapShuffleLocations; import scala.None$; import scala.Option; import scala.Product2; @@ -134,8 +136,11 @@ public void write(Iterator> records) throws IOException { try { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + Optional blockLocs = mapOutputWriter.commitAllPartitions(); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), + blockLocs.orNull(), + partitionLengths); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -168,8 +173,11 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + Optional mapLocations = mapOutputWriter.commitAllPartitions(); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), + mapLocations.orNull(), + partitionLengths); } catch (Exception e) { try { mapOutputWriter.abort(e); @@ -178,6 +186,10 @@ public void write(Iterator> records) throws IOException { } throw e; } + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), + DefaultMapShuffleLocations.get(blockManager.shuffleServerId()), + partitionLengths); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java new file mode 100644 index 000000000000..ffd97c0f2660 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java @@ -0,0 +1,76 @@ +/* + * 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.sort; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + +import org.apache.spark.api.shuffle.MapShuffleLocations; +import org.apache.spark.api.shuffle.ShuffleLocation; +import org.apache.spark.storage.BlockManagerId; + +import java.util.Objects; + +public class DefaultMapShuffleLocations implements MapShuffleLocations, ShuffleLocation { + + /** + * We borrow the cache size from the BlockManagerId's cache - around 1MB, which should be + * feasible. + */ + private static final LoadingCache + DEFAULT_SHUFFLE_LOCATIONS_CACHE = + CacheBuilder.newBuilder() + .maximumSize(BlockManagerId.blockManagerIdCacheSize()) + .build(new CacheLoader() { + @Override + public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { + return new DefaultMapShuffleLocations(blockManagerId); + } + }); + + private final BlockManagerId location; + + public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { + this.location = blockManagerId; + } + + public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { + return DEFAULT_SHUFFLE_LOCATIONS_CACHE.getUnchecked(blockManagerId); + } + + @Override + public ShuffleLocation getLocationForBlock(int reduceId) { + return this; + } + + public BlockManagerId getBlockManagerId() { + return location; + } + + @Override + public boolean equals(Object other) { + return other instanceof DefaultMapShuffleLocations + && Objects.equals(((DefaultMapShuffleLocations) other).location, location); + } + + @Override + public int hashCode() { + return Objects.hashCode(location); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index b5ca6c8a50ce..95c4577cb770 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -23,6 +23,8 @@ import java.nio.channels.WritableByteChannel; import java.util.Iterator; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.shuffle.MapShuffleLocations; import scala.Option; import scala.Product2; import scala.collection.JavaConverters; @@ -221,6 +223,7 @@ void closeAndWriteOutput() throws IOException { final ShuffleMapOutputWriter mapWriter = shuffleWriteSupport .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); final long[] partitionLengths; + Optional mapLocations; try { try { partitionLengths = mergeSpills(spills, mapWriter); @@ -231,7 +234,7 @@ void closeAndWriteOutput() throws IOException { } } } - mapWriter.commitAllPartitions(); + mapLocations = mapWriter.commitAllPartitions(); } catch (Exception e) { try { mapWriter.abort(e); @@ -240,7 +243,10 @@ void closeAndWriteOutput() throws IOException { } throw e; } - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); + mapStatus = MapStatus$.MODULE$.apply( + blockManager.shuffleServerId(), + mapLocations.orNull(), + partitionLengths); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java index 76e87a674025..f7ec202ef4b9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java @@ -46,6 +46,6 @@ public ShuffleWriteSupport writes() { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } - return new DefaultShuffleWriteSupport(sparkConf, blockResolver); + return new DefaultShuffleWriteSupport(sparkConf, blockResolver, blockManager.shuffleServerId()); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index c84158e1891d..7eb0d56776de 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -24,6 +24,10 @@ import java.io.OutputStream; import java.nio.channels.FileChannel; +import org.apache.spark.api.java.Optional; +import org.apache.spark.api.shuffle.MapShuffleLocations; +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations; +import org.apache.spark.storage.BlockManagerId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,6 +53,7 @@ public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { private final int bufferSize; private int currPartitionId = 0; private long currChannelPosition; + private final BlockManagerId shuffleServerId; private final File outputFile; private File outputTempFile; @@ -61,11 +66,13 @@ public DefaultShuffleMapOutputWriter( int shuffleId, int mapId, int numPartitions, + BlockManagerId shuffleServerId, ShuffleWriteMetricsReporter metrics, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; this.mapId = mapId; + this.shuffleServerId = shuffleServerId; this.metrics = metrics; this.blockResolver = blockResolver; this.bufferSize = @@ -90,10 +97,11 @@ public ShufflePartitionWriter getNextPartitionWriter() throws IOException { } @Override - public void commitAllPartitions() throws IOException { + public Optional commitAllPartitions() throws IOException { cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); + return Optional.of(DefaultMapShuffleLocations.get(shuffleServerId)); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java index f8fadd0ecfa6..86f158349568 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java @@ -22,17 +22,21 @@ import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; import org.apache.spark.api.shuffle.ShuffleWriteSupport; import org.apache.spark.shuffle.IndexShuffleBlockResolver; +import org.apache.spark.storage.BlockManagerId; public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { private final SparkConf sparkConf; private final IndexShuffleBlockResolver blockResolver; + private final BlockManagerId shuffleServerId; public DefaultShuffleWriteSupport( SparkConf sparkConf, - IndexShuffleBlockResolver blockResolver) { + IndexShuffleBlockResolver blockResolver, + BlockManagerId shuffleServerId) { this.sparkConf = sparkConf; this.blockResolver = blockResolver; + this.shuffleServerId = shuffleServerId; } @Override @@ -41,7 +45,7 @@ public ShuffleMapOutputWriter createMapOutputWriter( int mapId, int numPartitions) { return new DefaultShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, + shuffleId, mapId, numPartitions, shuffleServerId, TaskContext.get().taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5c820f5a2ed0..81a8001fde68 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -28,6 +28,7 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.util.control.NonFatal +import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -281,9 +282,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } // For testing - def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { - getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) + def getMapSizesByShuffleLocation(shuffleId: Int, reduceId: Int) + : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { + getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1) } /** @@ -295,8 +296,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * and the second item is a sequence of (shuffle block id, shuffle block size) tuples * describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] + def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -645,8 +646,8 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. - def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -682,12 +683,13 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private val fetching = new HashSet[Int] // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. - override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + override def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { - MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses( + shuffleId, startPartition, endPartition, statuses) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -871,9 +873,9 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus]): Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] + val splitsByAddress = new HashMap[ShuffleLocation, ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" @@ -883,7 +885,8 @@ private[spark] object MapOutputTracker extends Logging { for (part <- startPartition until endPartition) { val size = status.getSizeForBlock(part) if (size != 0) { - splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += + val shuffleLoc = status.mapShuffleLocations.getLocationForBlock(part) + splitsByAddress.getOrElseUpdate(shuffleLoc, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 64f0a060a247..a61f9bd14ef2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -24,7 +24,9 @@ import scala.collection.mutable import org.roaringbitmap.RoaringBitmap import org.apache.spark.SparkEnv +import org.apache.spark.api.shuffle.MapShuffleLocations import org.apache.spark.internal.config +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils @@ -33,7 +35,16 @@ import org.apache.spark.util.Utils * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. */ private[spark] sealed trait MapStatus { - /** Location where this task was run. */ + + /** + * Locations where this task stored shuffle blocks. + * + * May be null if the MapOutputTracker is not tracking the location of shuffle blocks, leaving it + * up to the implementation of shuffle plugins to do so. + */ + def mapShuffleLocations: MapShuffleLocations + + /** Location where the task was run. */ def location: BlockManagerId /** @@ -56,11 +67,31 @@ private[spark] object MapStatus { .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get) + // A temporary concession to the fact that we only expect implementations of shuffle provided by + // Spark to be storing shuffle locations in the driver, meaning we want to introduce as little + // serialization overhead as possible in such default cases. + // + // If more similar cases arise, consider adding a serialization API for these shuffle locations. + private val DEFAULT_MAP_SHUFFLE_LOCATIONS_ID: Byte = 0 + private val NON_DEFAULT_MAP_SHUFFLE_LOCATIONS_ID: Byte = 1 + + /** + * Visible for testing. + */ def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + apply(loc, DefaultMapShuffleLocations.get(loc), uncompressedSizes) + } + + def apply( + loc: BlockManagerId, + mapShuffleLocs: MapShuffleLocations, + uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus( + loc, mapShuffleLocs, uncompressedSizes) } else { - new CompressedMapStatus(loc, uncompressedSizes) + new CompressedMapStatus( + loc, mapShuffleLocs, uncompressedSizes) } } @@ -91,41 +122,89 @@ private[spark] object MapStatus { math.pow(LOG_BASE, compressedSize & 0xFF).toLong } } -} + def writeLocations( + loc: BlockManagerId, + mapShuffleLocs: MapShuffleLocations, + out: ObjectOutput): Unit = { + if (mapShuffleLocs != null) { + out.writeBoolean(true) + if (mapShuffleLocs.isInstanceOf[DefaultMapShuffleLocations] + && mapShuffleLocs.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId == loc) { + out.writeByte(MapStatus.DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) + } else { + out.writeByte(MapStatus.NON_DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) + out.writeObject(mapShuffleLocs) + } + } else { + out.writeBoolean(false) + } + loc.writeExternal(out) + } + + def readLocations(in: ObjectInput): (BlockManagerId, MapShuffleLocations) = { + if (in.readBoolean()) { + val locId = in.readByte() + if (locId == MapStatus.DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) { + val blockManagerId = BlockManagerId(in) + (blockManagerId, DefaultMapShuffleLocations.get(blockManagerId)) + } else { + val mapShuffleLocations = in.readObject().asInstanceOf[MapShuffleLocations] + val blockManagerId = BlockManagerId(in) + (blockManagerId, mapShuffleLocations) + } + } else { + val blockManagerId = BlockManagerId(in) + (blockManagerId, null) + } + } +} /** * A [[MapStatus]] implementation that tracks the size of each block. Size for each block is * represented using a single byte. * - * @param loc location where the task is being executed. + * @param loc Location were the task is being executed. + * @param mapShuffleLocs locations where the task stored its shuffle blocks - may be null. * @param compressedSizes size of the blocks, indexed by reduce partition id. */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, + private[this] var mapShuffleLocs: MapShuffleLocations, private[this] var compressedSizes: Array[Byte]) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + // For deserialization only + protected def this() = this(null, null, null.asInstanceOf[Array[Byte]]) - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.map(MapStatus.compressSize)) + def this( + loc: BlockManagerId, + mapShuffleLocations: MapShuffleLocations, + uncompressedSizes: Array[Long]) { + this( + loc, + mapShuffleLocations, + uncompressedSizes.map(MapStatus.compressSize)) } override def location: BlockManagerId = loc + override def mapShuffleLocations: MapShuffleLocations = mapShuffleLocs + override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - loc.writeExternal(out) + MapStatus.writeLocations(loc, mapShuffleLocs, out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - loc = BlockManagerId(in) + val (deserializedLoc, deserializedMapShuffleLocs) = MapStatus.readLocations(in) + loc = deserializedLoc + mapShuffleLocs = deserializedMapShuffleLocs val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) @@ -138,6 +217,7 @@ private[spark] class CompressedMapStatus( * plus a bitmap for tracking which blocks are empty. * * @param loc location where the task is being executed + * @param mapShuffleLocs location where the task stored shuffle blocks - may be null * @param numNonEmptyBlocks the number of non-empty blocks * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks @@ -145,6 +225,7 @@ private[spark] class CompressedMapStatus( */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, + private[this] var mapShuffleLocs: MapShuffleLocations, private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, @@ -155,10 +236,12 @@ private[spark] class HighlyCompressedMapStatus private ( require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1, null) // For deserialization only + protected def this() = this(null, null, -1, null, -1, null) // For deserialization only override def location: BlockManagerId = loc + override def mapShuffleLocations: MapShuffleLocations = mapShuffleLocs + override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -172,7 +255,7 @@ private[spark] class HighlyCompressedMapStatus private ( } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - loc.writeExternal(out) + MapStatus.writeLocations(loc, mapShuffleLocs, out) emptyBlocks.writeExternal(out) out.writeLong(avgSize) out.writeInt(hugeBlockSizes.size) @@ -183,7 +266,9 @@ private[spark] class HighlyCompressedMapStatus private ( } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - loc = BlockManagerId(in) + val (deserializedLoc, deserializedMapShuffleLocs) = MapStatus.readLocations(in) + loc = deserializedLoc + mapShuffleLocs = deserializedMapShuffleLocs emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() @@ -199,7 +284,10 @@ private[spark] class HighlyCompressedMapStatus private ( } private[spark] object HighlyCompressedMapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply( + loc: BlockManagerId, + mapShuffleLocs: MapShuffleLocations, + uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -239,7 +327,12 @@ private[spark] object HighlyCompressedMapStatus { } emptyBlocks.trim() emptyBlocks.runOptimize() - new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizes) + new HighlyCompressedMapStatus( + loc, + mapShuffleLocs, + numNonEmptyBlocks, + emptyBlocks, + avgSize, + hugeBlockSizes) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 20774c8d999c..48923666bab6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,7 +31,7 @@ import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSe import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} import com.esotericsoftware.kryo.pool.{KryoCallback, KryoFactory, KryoPool} -import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} +import com.esotericsoftware.kryo.serializers.{ExternalizableSerializer, JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} import org.roaringbitmap.RoaringBitmap @@ -151,6 +151,8 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[SerializableConfiguration], new KryoJavaSerializer()) kryo.register(classOf[SerializableJobConf], new KryoJavaSerializer()) kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) + kryo.register(classOf[CompressedMapStatus], new ExternalizableSerializer()) + kryo.register(classOf[HighlyCompressedMapStatus], new ExternalizableSerializer()) kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas)) kryo.register(classOf[GenericData.Record], new GenericAvroSerializer(avroSchemas)) @@ -454,8 +456,6 @@ private[serializer] object KryoSerializer { private val toRegister: Seq[Class[_]] = Seq( ByteBuffer.allocate(1).getClass, classOf[StorageLevel], - classOf[CompressedMapStatus], - classOf[HighlyCompressedMapStatus], classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Boolean]], diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index c7843710413d..4d559556360c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -20,7 +20,8 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer.SerializerManager -import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations +import org.apache.spark.storage.{BlockId, BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -47,7 +48,14 @@ private[spark] class BlockStoreShuffleReader[K, C]( context, blockManager.shuffleClient, blockManager, - mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), + mapOutputTracker.getMapSizesByShuffleLocation(handle.shuffleId, startPartition, endPartition) + .map { + case (loc: DefaultMapShuffleLocations, blocks: Seq[(BlockId, Long)]) => + (loc.getBlockManagerId, blocks) + case _ => + throw new UnsupportedOperationException("Not allowed to using non-default map shuffle" + + " locations yet.") + }, serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 62316f384b64..1fcae684b005 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,8 +67,11 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = writeSupport.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) val partitionLengths = sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + val mapLocations = mapOutputWriter.commitAllPartitions() + mapStatus = MapStatus( + blockManager.shuffleServerId, + mapLocations.orNull(), + partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index d188bdd912e5..97b99e08d9ca 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -132,12 +132,14 @@ private[spark] object BlockManagerId { getCachedBlockManagerId(obj) } + val blockManagerIdCacheSize = 10000 + /** * The max cache size is hardcoded to 10000, since the size of a BlockManagerId * object is about 48B, the total memory cost should be below 1MB which is feasible. */ val blockManagerIdCache = CacheBuilder.newBuilder() - .maximumSize(10000) + .maximumSize(blockManagerIdCacheSize) .build(new CacheLoader[BlockManagerId, BlockManagerId]() { override def load(id: BlockManagerId) = id }) diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index d37673aa69c3..2f73a6a90fe6 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -172,6 +172,8 @@ public void setUp() throws IOException { when(shuffleDep.serializer()).thenReturn(serializer); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); + when(blockManager.shuffleServerId()).thenReturn(BlockManagerId.apply( + "0", "localhost", 9099, Option.empty())); TaskContext$.MODULE$.setTaskContext(taskContext); } @@ -188,8 +190,7 @@ private UnsafeShuffleWriter createWriter( taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver) - ); + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver, blockManager.shuffleServerId())); } private void assertSpillFilesWereCleanedUp() { @@ -550,7 +551,7 @@ public void testPeakMemoryUsed() throws Exception { taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver, blockManager.shuffleServerId())); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index d86975964b55..0a77c4f6d583 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.internal.config.Network.{RPC_ASK_TIMEOUT, RPC_MESSAGE_MA import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} class MapOutputTrackerSuite extends SparkFunSuite { @@ -67,10 +68,13 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(1000L, 10000L))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(10000L, 1000L))) - val statuses = tracker.getMapSizesByExecutorId(10, 0) + val statuses = tracker.getMapSizesByShuffleLocation(10, 0) assert(statuses.toSet === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) + Seq( + (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), + ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), + (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)), + ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) .toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() @@ -90,11 +94,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) + assert(tracker.getMapSizesByShuffleLocation(10, 0).nonEmpty) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.unregisterShuffle(10) assert(!tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0).isEmpty) + assert(tracker.getMapSizesByShuffleLocation(10, 0).isEmpty) tracker.stop() rpcEnv.shutdown() @@ -121,7 +125,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the // stage already being aborted. - intercept[FetchFailedException] { tracker.getMapSizesByExecutorId(10, 1) } + intercept[FetchFailedException] { tracker.getMapSizesByShuffleLocation(10, 1) } tracker.stop() rpcEnv.shutdown() @@ -143,24 +147,26 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.registerShuffle(10, 1) slaveTracker.updateEpoch(masterTracker.getEpoch) // This is expected to fail because no outputs have been registered for the shuffle. - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("a", "hostA", 1000), Array(1000L))) slaveTracker.updateEpoch(masterTracker.getEpoch) - assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + assert(slaveTracker.getMapSizesByShuffleLocation(10, 0).toSeq === + Seq( + (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), + ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) assert(masterTracker.getEpoch > masterTrackerEpochBeforeLossOfMapOutput) slaveTracker.updateEpoch(masterTracker.getEpoch) - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } // failure should be cached - intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.stop() @@ -261,8 +267,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { // being sent. masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => + val bmId = BlockManagerId("999", "mps", 1000) masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) + bmId, + DefaultMapShuffleLocations.get(bmId), + Array.fill[Long](4000000)(0))) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -315,11 +324,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(size10000, size0, size1000, size0))) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === + assert(tracker.getMapSizesByShuffleLocation(10, 0, 4).toSeq === Seq( - (BlockManagerId("a", "hostA", 1000), + (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), Seq((ShuffleBlockId(10, 0, 1), size1000), (ShuffleBlockId(10, 0, 3), size10000))), - (BlockManagerId("b", "hostB", 1000), + (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)), Seq((ShuffleBlockId(10, 1, 0), size10000), (ShuffleBlockId(10, 1, 2), size1000))) ) ) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 90c790cefcca..83026c002f1b 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -73,7 +73,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // All blocks must have non-zero size (0 until NUM_BLOCKS).foreach { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) assert(statuses.forall(_._2.forall(blockIdSizePair => blockIdSizePair._2 > 0))) } } @@ -112,7 +112,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) statuses.flatMap(_._2.map(_._2)) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) @@ -137,7 +137,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) statuses.flatMap(_._2.map(_._2)) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index d58ee4e651e1..20b6dcd26966 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -29,12 +29,14 @@ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.api.shuffle.MapShuffleLocations import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} @@ -713,8 +715,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -740,8 +742,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", reduceRdd.partitions.length)))) // we can see both result blocks now - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === - HashSet("hostA", "hostB")) + assert(mapOutputTracker + .getMapSizesByShuffleLocation(shuffleId, 0) + .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .toSet === HashSet("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() @@ -779,11 +783,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(ExecutorLost("exec-hostA", event)) if (expectFileLoss) { intercept[MetadataFetchFailedException] { - mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) + mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0) } } else { - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) } } } @@ -1076,8 +1080,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) // The MapOutputTracker should know about both map output locations. - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === - HashSet("hostA", "hostB")) + assert(mapOutputTracker + .getMapSizesByShuffleLocation(shuffleId, 0) + .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .toSet === HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( @@ -1206,10 +1212,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // The MapOutputTracker should know about both map output locations. - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === - HashSet("hostA", "hostB")) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 1).map(_._1.host).toSet === - HashSet("hostA", "hostB")) + assert(mapOutputTracker + .getMapSizesByShuffleLocation(shuffleId, 0) + .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .toSet === HashSet("hostA", "hostB")) + assert(mapOutputTracker + .getMapSizesByShuffleLocation(shuffleId, 1) + .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) + .toSet === HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( @@ -1399,8 +1409,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi Success, makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostB"), makeShuffleLocation("hostA"))) // finish the next stage normally, which completes the job complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -1554,7 +1564,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi reduceIdx <- reduceIdxs } { // this would throw an exception if the map status hadn't been registered - val statuses = mapOutputTracker.getMapSizesByExecutorId(stage, reduceIdx) + val statuses = mapOutputTracker.getMapSizesByShuffleLocation(stage, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) @@ -1606,7 +1616,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // check that we have all the map output for stage 0 (0 until reduceRdd.partitions.length).foreach { reduceIdx => - val statuses = mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx) + val statuses = mapOutputTracker.getMapSizesByShuffleLocation(0, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) @@ -1805,8 +1815,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) // Make sure that the reduce stage was now submitted. assert(taskSets.size === 3) @@ -2068,8 +2078,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"))) // Reducer should run on the same host that map task ran val reduceTaskSet = taskSets(1) @@ -2114,8 +2124,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) - assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"))) // Reducer should run where RDD 2 has preferences, even though it also has a shuffle dep val reduceTaskSet = taskSets(1) @@ -2278,8 +2288,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) - assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) assert(listener1.results.size === 1) // When attempting the second stage, show a fetch failure @@ -2294,8 +2304,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(2).stageId === 0) complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) + assert(listener2.results.size === 0) // Second stage listener should still not have a result // Stage 1 should now be running as task set 3; make its first task succeed @@ -2303,8 +2314,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(3), Seq( (Success, makeMapStatus("hostB", rdd2.partitions.length)), (Success, makeMapStatus("hostD", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByExecutorId(dep2.shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostD"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(dep2.shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostB"), makeShuffleLocation("hostD"))) assert(listener2.results.size === 1) // Finally, the reduce job should be running as task set 4; make it see a fetch failure, @@ -2342,8 +2353,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) - assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) assert(listener1.results.size === 1) // When attempting stage1, trigger a fetch failure. @@ -2368,8 +2379,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(2).stageId === 0) complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === - Set(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) + assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === + Set(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) // After stage0 is finished, stage1 will be submitted and found there is no missing // partitions in it. Then listener got triggered. @@ -2982,6 +2993,10 @@ object DAGSchedulerSuite { def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) + + def makeShuffleLocation(host: String): MapShuffleLocations = { + DefaultMapShuffleLocations.get(makeBlockManagerId(host)) + } } object FailThisAttempt { diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index c1e7fb9a1db1..3c786c0927bc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.LocalSparkContext._ import org.apache.spark.internal.config import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId class MapStatusSuite extends SparkFunSuite { @@ -61,7 +62,11 @@ class MapStatusSuite extends SparkFunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val status = MapStatus(BlockManagerId("a", "b", 10), sizes) + val bmId = BlockManagerId("a", "b", 10) + val status = MapStatus( + bmId, + DefaultMapShuffleLocations.get(bmId), + sizes) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -75,7 +80,7 @@ class MapStatusSuite extends SparkFunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, sizes) + val status = MapStatus(null, null, sizes) assert(status.isInstanceOf[HighlyCompressedMapStatus]) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) @@ -86,11 +91,13 @@ class MapStatusSuite extends SparkFunSuite { test("HighlyCompressedMapStatus: estimated size should be the average non-empty block size") { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) - val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val bmId = BlockManagerId("a", "b", 10) + val loc = DefaultMapShuffleLocations.get(bmId) + val status = MapStatus(bmId, loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) - assert(status1.location == loc) + assert(status1.location == loc.getBlockManagerId) + assert(status1.mapShuffleLocations == loc) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -108,11 +115,13 @@ class MapStatusSuite extends SparkFunSuite { val sizes = (0L to 3000L).toArray val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length - val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + val bmId = BlockManagerId("a", "b", 10) + val loc = DefaultMapShuffleLocations.get(bmId) + val status = MapStatus(bmId, loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) - assert(status1.location == loc) + assert(status1.location === bmId) + assert(status1.mapShuffleLocations === loc) for (i <- 0 until threshold) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -165,7 +174,8 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray - val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) + val bmId = BlockManagerId("exec-0", "host-0", 100) + val status1 = MapStatus(bmId, DefaultMapShuffleLocations.get(bmId), sizes) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 577d77e890d7..ec828db9391d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -191,7 +191,8 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa shuffleId <- shuffleIds reduceIdx <- (0 until nParts) } { - val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceIdx) + val statuses = taskScheduler.mapOutputTracker.getMapSizesByShuffleLocation( + shuffleId, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 2442670b6d3f..63f6942dee18 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -36,8 +36,9 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.ThreadUtils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") @@ -350,8 +351,10 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) + val bmId = BlockManagerId("exec-1", "host", 1234) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) + ser.serialize(HighlyCompressedMapStatus( + bmId, DefaultMapShuffleLocations.get(bmId), blockSizes)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 6d2ef17a7a79..b3073addb7cc 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} +import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} /** @@ -102,14 +103,17 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Make a mocked MapOutputTracker for the shuffle reader to use to determine what // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) - when(mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1)).thenReturn { + when(mapOutputTracker.getMapSizesByShuffleLocation( + shuffleId, reduceId, reduceId + 1)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) (shuffleBlockId, byteOutputStream.size().toLong) } - Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator + Seq( + (DefaultMapShuffleLocations.get(localBlockManagerId), shuffleBlockIdsAndSizes)) + .toIterator } // Create a mocked shuffle handle to pass into HashShuffleReader. diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 60096edb801c..23bf9aaf41e1 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt, anyString} +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach @@ -129,7 +129,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte metricsSystem = null, taskMetrics = taskMetrics)) - writeSupport = new DefaultShuffleWriteSupport(conf, blockResolver) + writeSupport = new DefaultShuffleWriteSupport( + conf, blockResolver, BlockManagerId("0", "localhost", 7090)) } override def afterEach(): Unit = { @@ -192,7 +193,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(taskMetrics.memoryBytesSpilled === 0) } - // TODO(ifilonenko): MAKE THIS PASS test("write with some empty partitions with transferTo") { def records: Iterator[(Int, Int)] = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala index 22d52924a7c7..d704f72015ce 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -87,7 +88,13 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft }).when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) mapOutputWriter = new DefaultShuffleMapOutputWriter( - 0, 0, NUM_PARTITIONS, shuffleWriteMetrics, blockResolver, conf) + 0, + 0, + NUM_PARTITIONS, + BlockManagerId("0", "localhost", 9099), + shuffleWriteMetrics, + blockResolver, + conf) } private def readRecordsFromFile(fromByte: Boolean): Array[Array[Int]] = { From 3f0c131c4fb67bcf867149449e26734bda7b7538 Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 19 Apr 2019 15:09:34 -0700 Subject: [PATCH 06/32] [SPARK-25299] Move shuffle writers back to being given specific partition ids (#540) We originally made the shuffle map output writer API behave like an iterator in fetching the "next" partition writer. However, the shuffle writer implementations tend to skip opening empty partitions. If we used an iterator-like API though we would be tied down to opening a partition writer for every single partition, even if some of them are empty. Here, we go back to using specific partition identifiers to give us more freedom to avoid needing to create writers for empty partitions. --- .../api/shuffle/ShuffleMapOutputWriter.java | 2 +- .../sort/BypassMergeSortShuffleWriter.java | 2 +- .../shuffle/sort/UnsafeShuffleWriter.java | 16 ++------- .../io/DefaultShuffleMapOutputWriter.java | 10 ++++-- .../util/collection/ExternalSorter.scala | 36 ++----------------- .../DefaultShuffleMapOutputWriterSuite.scala | 8 ++--- 6 files changed, 17 insertions(+), 57 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java index 181701175d35..062cf4ff0fba 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java @@ -30,7 +30,7 @@ */ @Experimental public interface ShuffleMapOutputWriter { - ShufflePartitionWriter getNextPartitionWriter() throws IOException; + ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException; Optional commitAllPartitions() throws IOException; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 434286175e41..b339738de5ad 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -216,7 +216,7 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro boolean copyThrewException = true; ShufflePartitionWriter writer = null; try { - writer = mapOutputWriter.getNextPartitionWriter(); + writer = mapOutputWriter.getPartitionWriter(i); if (!file.exists()) { copyThrewException = false; } else { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 95c4577cb770..e4175f985d91 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -291,18 +291,6 @@ private long[] mergeSpills(SpillInfo[] spills, long[] partitionLengths = new long[numPartitions]; try { if (spills.length == 0) { - // The contract we are working under states that we will open a partition writer for - // each partition, regardless of number of spills - for (int i = 0; i < numPartitions; i++) { - ShufflePartitionWriter writer = null; - try { - writer = mapWriter.getNextPartitionWriter(); - } finally { - if (writer != null) { - writer.close(); - } - } - } return partitionLengths; } else { // There are multiple spills to merge, so none of these spill files' lengths were counted @@ -378,7 +366,7 @@ private long[] mergeSpillsWithFileStream( boolean copyThrewExecption = true; ShufflePartitionWriter writer = null; try { - writer = mapWriter.getNextPartitionWriter(); + writer = mapWriter.getPartitionWriter(partition); OutputStream partitionOutput = null; try { // Shield the underlying output stream from close() calls, so that we can close the @@ -457,7 +445,7 @@ private long[] mergeSpillsWithTransferTo( boolean copyThrewExecption = true; ShufflePartitionWriter writer = null; try { - writer = mapWriter.getNextPartitionWriter(); + writer = mapWriter.getPartitionWriter(partition); WritableByteChannel channel = writer.toChannel(); for (int i = 0; i < spills.length; i++) { long partitionLengthInSpill = 0L; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index 7eb0d56776de..926c3b943399 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -51,7 +51,7 @@ public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; - private int currPartitionId = 0; + private int lastPartitionId = -1; private long currChannelPosition; private final BlockManagerId shuffleServerId; @@ -84,7 +84,11 @@ public DefaultShuffleMapOutputWriter( } @Override - public ShufflePartitionWriter getNextPartitionWriter() throws IOException { + public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException { + if (partitionId <= lastPartitionId) { + throw new IllegalArgumentException("Partitions should be requested in increasing order."); + } + lastPartitionId = partitionId; if (outputTempFile == null) { outputTempFile = Utils.tempFileWith(outputFile); } @@ -93,7 +97,7 @@ public ShufflePartitionWriter getNextPartitionWriter() throws IOException { } else { currChannelPosition = 0L; } - return new DefaultShufflePartitionWriter(currPartitionId++); + return new DefaultShufflePartitionWriter(partitionId); } @Override diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 445db7cac60c..6829103c7da7 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -717,17 +717,6 @@ private[spark] class ExternalSorter[K, V, C]( lengths } - private def writeEmptyPartition(mapOutputWriter: ShuffleMapOutputWriter): Unit = { - var partitionWriter: ShufflePartitionWriter = null - try { - partitionWriter = mapOutputWriter.getNextPartitionWriter - } finally { - if (partitionWriter != null) { - partitionWriter.close() - } - } - } - /** * Write all the data added into this ExternalSorter into a map output writer that pushes bytes * to some arbitrary backing store. This is called by the SortShuffleWriter. @@ -738,26 +727,16 @@ private[spark] class ExternalSorter[K, V, C]( shuffleId: Int, mapId: Int, mapOutputWriter: ShuffleMapOutputWriter): Array[Long] = { // Track location of each range in the map output val lengths = new Array[Long](numPartitions) - var nextPartitionId = 0 if (spills.isEmpty) { // Case where we only have in-memory data val collection = if (aggregator.isDefined) map else buffer val it = collection.destructiveSortedWritablePartitionedIterator(comparator) while (it.hasNext()) { val partitionId = it.nextPartition() - // The contract for the plugin is that we will ask for a writer for every partition - // even if it's empty. However, the external sorter will return non-contiguous - // partition ids. So this loop "backfills" the empty partitions that form the gaps. - - // The algorithm as a whole is correct because the partition ids are returned by the - // iterator in ascending order. - for (emptyPartition <- nextPartitionId until partitionId) { - writeEmptyPartition(mapOutputWriter) - } var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null try { - partitionWriter = mapOutputWriter.getNextPartitionWriter + partitionWriter = mapOutputWriter.getPartitionWriter(partitionId) val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) partitionPairsWriter = new ShufflePartitionPairsWriter( partitionWriter, @@ -779,7 +758,6 @@ private[spark] class ExternalSorter[K, V, C]( if (partitionWriter != null) { lengths(partitionId) = partitionWriter.getNumBytesWritten } - nextPartitionId = partitionId + 1 } } else { // We must perform merge-sort; get an iterator by partition and write everything directly. @@ -790,14 +768,11 @@ private[spark] class ExternalSorter[K, V, C]( // The algorithm as a whole is correct because the partition ids are returned by the // iterator in ascending order. - for (emptyPartition <- nextPartitionId until id) { - writeEmptyPartition(mapOutputWriter) - } val blockId = ShuffleBlockId(shuffleId, mapId, id) var partitionWriter: ShufflePartitionWriter = null var partitionPairsWriter: ShufflePartitionPairsWriter = null try { - partitionWriter = mapOutputWriter.getNextPartitionWriter + partitionWriter = mapOutputWriter.getPartitionWriter(id) partitionPairsWriter = new ShufflePartitionPairsWriter( partitionWriter, serializerManager, @@ -817,16 +792,9 @@ private[spark] class ExternalSorter[K, V, C]( if (partitionWriter != null) { lengths(id) = partitionWriter.getNumBytesWritten } - nextPartitionId = id + 1 } } - // The iterator may have stopped short of opening a writer for every partition. So fill in the - // remaining empty partitions. - for (emptyPartition <- nextPartitionId until numPartitions) { - writeEmptyPartition(mapOutputWriter) - } - context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala index d704f72015ce..420b0d4d2f67 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala @@ -140,7 +140,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("writing to an outputstream") { (0 until NUM_PARTITIONS).foreach{ p => - val writer = mapOutputWriter.getNextPartitionWriter + val writer = mapOutputWriter.getPartitionWriter(p) val stream = writer.toStream() data(p).foreach { i => stream.write(i)} stream.close() @@ -159,7 +159,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("writing to a channel") { (0 until NUM_PARTITIONS).foreach{ p => - val writer = mapOutputWriter.getNextPartitionWriter + val writer = mapOutputWriter.getPartitionWriter(p) val channel = writer.toChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() @@ -179,7 +179,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("copyStreams with an outputstream") { (0 until NUM_PARTITIONS).foreach{ p => - val writer = mapOutputWriter.getNextPartitionWriter + val writer = mapOutputWriter.getPartitionWriter(p) val stream = writer.toStream() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() @@ -200,7 +200,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("copyStreamsWithNIO with a channel") { (0 until NUM_PARTITIONS).foreach{ p => - val writer = mapOutputWriter.getNextPartitionWriter + val writer = mapOutputWriter.getPartitionWriter(p) val channel = writer.toChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() From f982df7cc1e114fc1ab9b309482fb49ea388a9c6 Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 19 Apr 2019 15:19:57 -0700 Subject: [PATCH 07/32] [SPARK-25299] Don't set map status twice in bypass merge sort shuffle writer (#541) --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index b339738de5ad..22386c39aca0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -186,10 +186,6 @@ public void write(Iterator> records) throws IOException { } throw e; } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), - DefaultMapShuffleLocations.get(blockManager.shuffleServerId()), - partitionLengths); } @VisibleForTesting From 6891197d10ef09cd89058f5c72206114771b9262 Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 24 May 2019 12:15:03 -0700 Subject: [PATCH 08/32] [SPARK-25299] Propose a new NIO transfer API for partition writing. (#535) * Propose a new NIO transfer API for partition writing. This solves the consistency and resource leakage concerns with the first iteration of thie API, where it would not be obvious that the streamable resources created by ShufflePartitionWriter needed to be closed by ShuffleParittionWriter#close as opposed to closing the resources directly. This introduces the following adjustments: - Channel-based writes are separated out to their own module, SupportsTransferTo. This allows the transfer-to APIs to be modified independently, and users that only provide output streams can ignore the NIO APIs entirely. This also allows us to mark the base ShufflePartitionWriter as a stable API eventually while keeping the NIO APIs marked as experimental or developer-api. - We add APIs that explicitly encodes the notion of transferring bytes from one source to another. The partition writer returns an instance of TransferrableWritableByteChannel, which has APIs for accepting a TransferrableReadableByteChannel and can tell the readable byte channel to transfer its bytes out to some destination sink. - The resources returned by ShufflePartitionWriter are always closed. Internally, DefaultMapOutputWriter keeps resources open until commitAllPartitions() is called. * Migrate unsafe shuffle writer to use new byte channel API. * More sane implementation for unsafe * Fix style * Address comments * Fix imports * Fix build * Fix more build problems * Address comments. --- .../api/shuffle/ShufflePartitionWriter.java | 42 ++------ .../api/shuffle/ShuffleWriteSupport.java | 2 +- .../spark/api/shuffle/SupportsTransferTo.java | 53 +++++++++ .../TransferrableWritableByteChannel.java | 54 ++++++++++ .../sort/BypassMergeSortShuffleWriter.java | 79 +++++++------- ...faultTransferrableWritableByteChannel.java | 51 +++++++++ .../shuffle/sort/UnsafeShuffleWriter.java | 94 ++++++++-------- .../io/DefaultShuffleMapOutputWriter.java | 102 +++++++++++------- .../shuffle/sort/SortShuffleManager.scala | 2 - .../util/collection/ExternalSorter.scala | 3 - .../ShufflePartitionPairsWriter.scala | 16 +-- .../sort/UnsafeShuffleWriterSuite.java | 6 +- .../BypassMergeSortShuffleWriterSuite.scala | 5 - .../DefaultShuffleMapOutputWriterSuite.scala | 32 +++--- 14 files changed, 336 insertions(+), 205 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java create mode 100644 core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java index 6a53803e5d11..74c928b0b9c8 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java @@ -17,13 +17,10 @@ package org.apache.spark.api.shuffle; -import java.io.Closeable; import java.io.IOException; import java.io.OutputStream; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import org.apache.http.annotation.Experimental; +import org.apache.spark.annotation.Experimental; /** * :: Experimental :: @@ -32,43 +29,16 @@ * @since 3.0.0 */ @Experimental -public interface ShufflePartitionWriter extends Closeable { +public interface ShufflePartitionWriter { /** - * Returns an underlying {@link OutputStream} that can write bytes to the underlying data store. - *

- * Note that this stream itself is not closed by the caller; close the stream in the - * implementation of this interface's {@link #close()}. + * Opens and returns an underlying {@link OutputStream} that can write bytes to the underlying + * data store. */ - OutputStream toStream() throws IOException; + OutputStream openStream() throws IOException; /** - * Returns an underlying {@link WritableByteChannel} that can write bytes to the underlying data - * store. - *

- * Note that this channel itself is not closed by the caller; close the channel in the - * implementation of this interface's {@link #close()}. - */ - default WritableByteChannel toChannel() throws IOException { - return Channels.newChannel(toStream()); - } - - /** - * Get the number of bytes written by this writer's stream returned by {@link #toStream()} or - * the channel returned by {@link #toChannel()}. + * Get the number of bytes written by this writer's stream returned by {@link #openStream()}. */ long getNumBytesWritten(); - - /** - * Close all resources created by this ShufflePartitionWriter, via calls to {@link #toStream()} - * or {@link #toChannel()}. - *

- * This must always close any stream returned by {@link #toStream()}. - *

- * Note that the default version of {@link #toChannel()} returns a {@link WritableByteChannel} - * that does not itself need to be closed up front; only the underlying output stream given by - * {@link #toStream()} must be closed. - */ - @Override - void close() throws IOException; } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java index 6c69d5db9fd0..7e2b6cf4133f 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.http.annotation.Experimental; +import org.apache.spark.annotation.Experimental; /** * :: Experimental :: diff --git a/core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java b/core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java new file mode 100644 index 000000000000..866b61d0bafd --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java @@ -0,0 +1,53 @@ +/* + * 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.api.shuffle; + +import java.io.IOException; + +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * Indicates that partition writers can transfer bytes directly from input byte channels to + * output channels that stream data to the underlying shuffle partition storage medium. + *

+ * This API is separated out for advanced users because it only needs to be used for + * specific low-level optimizations. The idea is that the returned channel can transfer bytes + * from the input file channel out to the backing storage system without copying data into + * memory. + *

+ * Most shuffle plugin implementations should use {@link ShufflePartitionWriter} instead. + * + * @since 3.0.0 + */ +@Experimental +public interface SupportsTransferTo extends ShufflePartitionWriter { + + /** + * Opens and returns a {@link TransferrableWritableByteChannel} for transferring bytes from + * input byte channels to the underlying shuffle data store. + */ + TransferrableWritableByteChannel openTransferrableChannel() throws IOException; + + /** + * Returns the number of bytes written either by this writer's output stream opened by + * {@link #openStream()} or the byte channel opened by {@link #openTransferrableChannel()}. + */ + @Override + long getNumBytesWritten(); +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java new file mode 100644 index 000000000000..18234d7c4c94 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java @@ -0,0 +1,54 @@ +/* + * 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.api.shuffle; + +import java.io.Closeable; +import java.io.IOException; + +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; +import org.apache.spark.annotation.Experimental; + +/** + * :: Experimental :: + * Represents an output byte channel that can copy bytes from input file channels to some + * arbitrary storage system. + *

+ * This API is provided for advanced users who can transfer bytes from a file channel to + * some output sink without copying data into memory. Most users should not need to use + * this functionality; this is primarily provided for the built-in shuffle storage backends + * that persist shuffle files on local disk. + *

+ * For a simpler alternative, see {@link ShufflePartitionWriter}. + * + * @since 3.0.0 + */ +@Experimental +public interface TransferrableWritableByteChannel extends Closeable { + + /** + * Copy all bytes from the source readable byte channel into this byte channel. + * + * @param source File to transfer bytes from. Do not call anything on this channel other than + * {@link FileChannel#transferTo(long, long, WritableByteChannel)}. + * @param transferStartPosition Start position of the input file to transfer from. + * @param numBytesToTransfer Number of bytes to transfer from the given source. + */ + void transferFrom(FileChannel source, long transferStartPosition, long numBytesToTransfer) + throws IOException; +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 22386c39aca0..128b90429209 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -21,12 +21,10 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.OutputStream; +import java.nio.channels.Channels; import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; import javax.annotation.Nullable; -import org.apache.spark.api.java.Optional; -import org.apache.spark.api.shuffle.MapShuffleLocations; import scala.None$; import scala.Option; import scala.Product2; @@ -38,19 +36,22 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +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; import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; import org.apache.spark.internal.config.package$; -import org.apache.spark.Partitioner; -import org.apache.spark.ShuffleDependency; -import org.apache.spark.SparkConf; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -90,7 +91,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int mapId; private final Serializer serializer; private final ShuffleWriteSupport shuffleWriteSupport; - private final IndexShuffleBlockResolver shuffleBlockResolver; /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; @@ -107,7 +107,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BypassMergeSortShuffleWriter( BlockManager blockManager, - IndexShuffleBlockResolver shuffleBlockResolver, BypassMergeSortShuffleHandle handle, int mapId, SparkConf conf, @@ -124,7 +123,6 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.numPartitions = partitioner.numPartitions(); this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); - this.shuffleBlockResolver = shuffleBlockResolver; this.shuffleWriteSupport = shuffleWriteSupport; } @@ -209,40 +207,43 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro try { for (int i = 0; i < numPartitions; i++) { final File file = partitionWriterSegments[i].file(); - boolean copyThrewException = true; - ShufflePartitionWriter writer = null; - try { - writer = mapOutputWriter.getPartitionWriter(i); - if (!file.exists()) { - copyThrewException = false; - } else { - if (transferToEnabled) { - WritableByteChannel outputChannel = writer.toChannel(); - FileInputStream in = new FileInputStream(file); - try (FileChannel inputChannel = in.getChannel()) { - Utils.copyFileStreamNIO(inputChannel, outputChannel, 0, inputChannel.size()); - copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); - } - } else { - OutputStream tempOutputStream = writer.toStream(); - FileInputStream in = new FileInputStream(file); - try { - Utils.copyStream(in, tempOutputStream, false, false); - copyThrewException = false; - } finally { - Closeables.close(in, copyThrewException); + ShufflePartitionWriter writer = mapOutputWriter.getPartitionWriter(i); + if (file.exists()) { + boolean copyThrewException = true; + if (transferToEnabled) { + FileInputStream in = new FileInputStream(file); + TransferrableWritableByteChannel outputChannel = null; + try (FileChannel inputChannel = in.getChannel()) { + if (writer instanceof SupportsTransferTo) { + outputChannel = ((SupportsTransferTo) writer).openTransferrableChannel(); + } else { + // Use default transferrable writable channel anyways in order to have parity with + // UnsafeShuffleWriter. + outputChannel = new DefaultTransferrableWritableByteChannel( + Channels.newChannel(writer.openStream())); } + outputChannel.transferFrom(inputChannel, 0L, inputChannel.size()); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + Closeables.close(outputChannel, copyThrewException); } - if (!file.delete()) { - logger.error("Unable to delete file for partition {}", i); + } else { + FileInputStream in = new FileInputStream(file); + OutputStream outputStream = null; + try { + outputStream = writer.openStream(); + Utils.copyStream(in, outputStream, false, false); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + Closeables.close(outputStream, copyThrewException); } } - } finally { - Closeables.close(writer, copyThrewException); + if (!file.delete()) { + logger.error("Unable to delete file for partition {}", i); + } } - lengths[i] = writer.getNumBytesWritten(); } } finally { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java new file mode 100644 index 000000000000..64ce851e392d --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java @@ -0,0 +1,51 @@ +/* + * 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.sort; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; +import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; +import org.apache.spark.util.Utils; + +/** + * This is used when transferTo is enabled but the shuffle plugin hasn't implemented + * {@link org.apache.spark.api.shuffle.SupportsTransferTo}. + *

+ * This default implementation exists as a convenience to the unsafe shuffle writer and + * the bypass merge sort shuffle writers. + */ +public class DefaultTransferrableWritableByteChannel implements TransferrableWritableByteChannel { + + private final WritableByteChannel delegate; + + public DefaultTransferrableWritableByteChannel(WritableByteChannel delegate) { + this.delegate = delegate; + } + + @Override + public void transferFrom( + FileChannel source, long transferStartPosition, long numBytesToTransfer) { + Utils.copyFileStreamNIO(source, delegate, transferStartPosition, numBytesToTransfer); + } + + @Override + public void close() throws IOException { + delegate.close(); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index e4175f985d91..a30345a93139 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -17,14 +17,12 @@ package org.apache.spark.shuffle.sort; +import java.nio.channels.Channels; import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; import java.util.Iterator; -import org.apache.spark.api.java.Optional; -import org.apache.spark.api.shuffle.MapShuffleLocations; import scala.Option; import scala.Product2; import scala.collection.JavaConverters; @@ -39,14 +37,17 @@ 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; import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.api.shuffle.SupportsTransferTo; import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; -import org.apache.commons.io.output.CloseShieldOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; @@ -54,11 +55,9 @@ import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; import org.apache.spark.unsafe.Platform; -import org.apache.spark.util.Utils; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -71,7 +70,6 @@ public class UnsafeShuffleWriter extends ShuffleWriter { static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; - private final IndexShuffleBlockResolver shuffleBlockResolver; private final TaskMemoryManager memoryManager; private final SerializerInstance serializer; private final Partitioner partitioner; @@ -107,7 +105,6 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream public UnsafeShuffleWriter( BlockManager blockManager, - IndexShuffleBlockResolver shuffleBlockResolver, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, int mapId, @@ -123,7 +120,6 @@ public UnsafeShuffleWriter( " reduce partitions"); } this.blockManager = blockManager; - this.shuffleBlockResolver = shuffleBlockResolver; this.memoryManager = memoryManager; this.mapId = mapId; final ShuffleDependency dep = handle.dependency(); @@ -364,45 +360,37 @@ private long[] mergeSpillsWithFileStream( } for (int partition = 0; partition < numPartitions; partition++) { boolean copyThrewExecption = true; - ShufflePartitionWriter writer = null; + ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); + OutputStream partitionOutput = null; try { - writer = mapWriter.getPartitionWriter(partition); - OutputStream partitionOutput = null; - try { - // Shield the underlying output stream from close() calls, so that we can close the - // higher level streams to make sure all data is really flushed and internal state - // is cleaned - partitionOutput = new CloseShieldOutputStream(writer.toStream()); - partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); - if (compressionCodec != null) { - partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); - } - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - - if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = null; - try { - partitionInputStream = new LimitedInputStream(spillInputStreams[i], - partitionLengthInSpill, false); - partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionOutput = writer.openStream(); + partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); + if (compressionCodec != null) { + partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); + } + for (int i = 0; i < spills.length; i++) { + final long partitionLengthInSpill = spills[i].partitionLengths[partition]; + + if (partitionLengthInSpill > 0) { + InputStream partitionInputStream = null; + try { + partitionInputStream = new LimitedInputStream(spillInputStreams[i], + partitionLengthInSpill, false); + partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionInputStream); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream( partitionInputStream); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream( - partitionInputStream); - } - ByteStreams.copy(partitionInputStream, partitionOutput); - } finally { - partitionInputStream.close(); } + ByteStreams.copy(partitionInputStream, partitionOutput); + } finally { + partitionInputStream.close(); } - copyThrewExecption = false; } - } finally { - Closeables.close(partitionOutput, copyThrewExecption); + copyThrewExecption = false; } } finally { - Closeables.close(writer, copyThrewExecption); + Closeables.close(partitionOutput, copyThrewExecption); } long numBytesWritten = writer.getNumBytesWritten(); partitionLengths[partition] = numBytesWritten; @@ -443,26 +431,34 @@ private long[] mergeSpillsWithTransferTo( } for (int partition = 0; partition < numPartitions; partition++) { boolean copyThrewExecption = true; - ShufflePartitionWriter writer = null; + ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); + TransferrableWritableByteChannel partitionChannel = null; try { +<<<<<<< HEAD writer = mapWriter.getPartitionWriter(partition); WritableByteChannel channel = writer.toChannel(); +||||||| parent of ca4448ec6d... [SPARK-25299] Propose a new NIO transfer API for partition writing. (#535) + writer = mapWriter.getNextPartitionWriter(); + WritableByteChannel channel = writer.toChannel(); +======= + partitionChannel = writer instanceof SupportsTransferTo ? + ((SupportsTransferTo) writer).openTransferrableChannel() + : new DefaultTransferrableWritableByteChannel( + Channels.newChannel(writer.openStream())); +>>>>>>> ca4448ec6d... [SPARK-25299] Propose a new NIO transfer API for partition writing. (#535) for (int i = 0; i < spills.length; i++) { long partitionLengthInSpill = 0L; partitionLengthInSpill += spills[i].partitionLengths[partition]; final FileChannel spillInputChannel = spillInputChannels[i]; final long writeStartTime = System.nanoTime(); - Utils.copyFileStreamNIO( - spillInputChannel, - channel, - spillInputChannelPositions[i], - partitionLengthInSpill); - copyThrewExecption = false; + partitionChannel.transferFrom( + spillInputChannel, spillInputChannelPositions[i], partitionLengthInSpill); spillInputChannelPositions[i] += partitionLengthInSpill; writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); } + copyThrewExecption = false; } finally { - Closeables.close(writer, copyThrewExecption); + Closeables.close(partitionChannel, copyThrewExecption); } long numBytes = writer.getNumBytesWritten(); partitionLengths[partition] = numBytes; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index 926c3b943399..e83db4e4bcef 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -24,18 +24,21 @@ import java.io.OutputStream; import java.nio.channels.FileChannel; -import org.apache.spark.api.java.Optional; -import org.apache.spark.api.shuffle.MapShuffleLocations; -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations; -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.shuffle.ShuffleWriteMetricsReporter; +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; @@ -151,70 +154,70 @@ private void initChannel() throws IOException { } } - private class DefaultShufflePartitionWriter implements ShufflePartitionWriter { + private class DefaultShufflePartitionWriter implements SupportsTransferTo { private final int partitionId; - private PartitionWriterStream stream = null; + private PartitionWriterStream partStream = null; + private PartitionWriterChannel partChannel = null; private DefaultShufflePartitionWriter(int partitionId) { this.partitionId = partitionId; } @Override - public OutputStream toStream() throws IOException { - if (outputFileChannel != null) { - throw new IllegalStateException("Requested an output channel for a previous write but" + - " now an output stream has been requested. Should not be using both channels" + - " and streams to write."); + public OutputStream openStream() throws IOException { + if (partStream == null) { + if (outputFileChannel != null) { + throw new IllegalStateException("Requested an output channel for a previous write but" + + " now an output stream has been requested. Should not be using both channels" + + " and streams to write."); + } + initStream(); + partStream = new PartitionWriterStream(partitionId); } - initStream(); - stream = new PartitionWriterStream(); - return stream; + return partStream; } @Override - public FileChannel toChannel() throws IOException { - if (stream != null) { - throw new IllegalStateException("Requested an output stream for a previous write but" + - " now an output channel has been requested. Should not be using both channels" + - " and streams to write."); + public TransferrableWritableByteChannel openTransferrableChannel() throws IOException { + if (partChannel == null) { + if (partStream != null) { + throw new IllegalStateException("Requested an output stream for a previous write but" + + " now an output channel has been requested. Should not be using both channels" + + " and streams to write."); + } + initChannel(); + partChannel = new PartitionWriterChannel(partitionId); } - initChannel(); - return outputFileChannel; + return partChannel; } @Override public long getNumBytesWritten() { - if (outputFileChannel != null && stream == null) { + if (partChannel != null) { try { - long newPosition = outputFileChannel.position(); - return newPosition - currChannelPosition; - } catch (Exception e) { - log.error("The partition which failed is: {}", partitionId, e); - throw new IllegalStateException("Failed to calculate position of file channel", e); + return partChannel.getCount(); + } catch (IOException e) { + throw new RuntimeException(e); } - } else if (stream != null) { - return stream.getCount(); + } else if (partStream != null) { + return partStream.getCount(); } else { // Assume an empty partition if stream and channel are never created return 0; } } - - @Override - public void close() { - if (stream != null) { - // Closing is a no-op. - stream.close(); - } - partitionLengths[partitionId] = getNumBytesWritten(); - } } private class PartitionWriterStream extends OutputStream { + private final int partitionId; private int count = 0; private boolean isClosed = false; + PartitionWriterStream(int partitionId) { + this.partitionId = partitionId; + } + public int getCount() { return count; } @@ -236,6 +239,7 @@ public void write(byte[] buf, int pos, int length) throws IOException { @Override public void close() { isClosed = true; + partitionLengths[partitionId] = count; } private void verifyNotClosed() { @@ -244,4 +248,24 @@ private void verifyNotClosed() { } } } + + private class PartitionWriterChannel extends DefaultTransferrableWritableByteChannel { + + private final int partitionId; + + PartitionWriterChannel(int partitionId) { + super(outputFileChannel); + this.partitionId = partitionId; + } + + public long getCount() throws IOException { + long writtenPosition = outputFileChannel.position(); + return writtenPosition - currChannelPosition; + } + + @Override + public void close() throws IOException { + partitionLengths[partitionId] = getCount(); + } + } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 849050556c56..5fa9296b022c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -140,7 +140,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => new UnsafeShuffleWriter( env.blockManager, - shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], context.taskMemoryManager(), unsafeShuffleHandle, mapId, @@ -151,7 +150,6 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new BypassMergeSortShuffleWriter( env.blockManager, - shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], bypassMergeSortHandle, mapId, env.conf, diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6829103c7da7..13132c2801ed 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -751,9 +751,6 @@ private[spark] class ExternalSorter[K, V, C]( if (partitionPairsWriter != null) { partitionPairsWriter.close() } - if (partitionWriter != null) { - partitionWriter.close() - } } if (partitionWriter != null) { lengths(partitionId) = partitionWriter.getNumBytesWritten diff --git a/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala index 6f19a2323efd..8538a78b377c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala @@ -54,25 +54,17 @@ private[spark] class ShufflePartitionPairsWriter( } private def open(): Unit = { - // The contract is that the partition writer is expected to close its own streams, but - // the compressor will only flush the stream when it is specifically closed. So we want to - // close objOut to flush the compressed bytes to the partition writer stream, but we don't want - // to close the partition output stream in the process. - partitionStream = new CloseShieldOutputStream(partitionWriter.toStream) + partitionStream = partitionWriter.openStream wrappedStream = serializerManager.wrapStream(blockId, partitionStream) objOut = serializerInstance.serializeStream(wrappedStream) } override def close(): Unit = { if (isOpen) { - // Closing objOut should propagate close to all inner layers - // We can't close wrappedStream explicitly because closing objOut and closing wrappedStream - // causes problems when closing compressed output streams twice. objOut.close() objOut = null wrappedStream = null partitionStream = null - partitionWriter.close() isOpen = false updateBytesWritten() } @@ -96,10 +88,4 @@ private[spark] class ShufflePartitionPairsWriter( writeMetrics.incBytesWritten(bytesWrittenDiff) curNumBytesWritten = numBytesWritten } - - private class CloseShieldOutputStream(delegate: OutputStream) - extends FilterOutputStream(delegate) { - - override def close(): Unit = flush() - } } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 2f73a6a90fe6..3c172a027ca0 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -183,8 +183,7 @@ private UnsafeShuffleWriter createWriter( conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter<>( blockManager, - shuffleBlockResolver, - taskMemoryManager, + taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, @@ -544,8 +543,7 @@ public void testPeakMemoryUsed() throws Exception { final UnsafeShuffleWriter writer = new UnsafeShuffleWriter<>( blockManager, - shuffleBlockResolver, - taskMemoryManager, + taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 23bf9aaf41e1..c491f39249c8 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -147,7 +147,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte test("write empty iterator") { val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, - blockResolver, shuffleHandle, 0, // MapId conf, @@ -173,7 +172,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, - blockResolver, shuffleHandle, 0, // MapId transferConf, @@ -198,7 +196,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, - blockResolver, shuffleHandle, 0, // MapId conf, @@ -234,7 +231,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, - blockResolver, shuffleHandle, 0, // MapId conf, @@ -257,7 +253,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte test("cleanup of intermediate files after errors") { val writer = new BypassMergeSortShuffleWriter[Int, Int]( blockManager, - blockResolver, shuffleHandle, 0, // MapId conf, diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala index 420b0d4d2f67..1f4ef0f20399 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle.sort.io import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} import java.math.BigInteger import java.nio.ByteBuffer +import java.nio.channels.{Channels, WritableByteChannel} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} @@ -31,10 +32,12 @@ import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.api.shuffle.SupportsTransferTo import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @@ -141,14 +144,13 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("writing to an outputstream") { (0 until NUM_PARTITIONS).foreach{ p => val writer = mapOutputWriter.getPartitionWriter(p) - val stream = writer.toStream() + val stream = writer.openStream() data(p).foreach { i => stream.write(i)} stream.close() intercept[IllegalStateException] { stream.write(p) } assert(writer.getNumBytesWritten() == D_LEN) - writer.close } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => D_LEN.toDouble}.toArray @@ -160,15 +162,23 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("writing to a channel") { (0 until NUM_PARTITIONS).foreach{ p => val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.toChannel() + val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() intBuffer.put(data(p)) - assert(channel.isOpen) - channel.write(byteBuffer) + val numBytes = byteBuffer.remaining() + val outputTempFile = File.createTempFile("channelTemp", "", tempDir) + val outputTempFileStream = new FileOutputStream(outputTempFile) + Utils.copyStream( + new ByteBufferInputStream(byteBuffer), + outputTempFileStream, + closeStreams = true) + val tempFileInput = new FileInputStream(outputTempFile) + channel.transferFrom(tempFileInput.getChannel, 0L, numBytes) // Bytes require * 4 + channel.close() + tempFileInput.close() assert(writer.getNumBytesWritten == D_LEN * 4) - writer.close } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray @@ -180,7 +190,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("copyStreams with an outputstream") { (0 until NUM_PARTITIONS).foreach{ p => val writer = mapOutputWriter.getPartitionWriter(p) - val stream = writer.toStream() + val stream = writer.openStream() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() intBuffer.put(data(p)) @@ -189,7 +199,6 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft in.close() stream.close() assert(writer.getNumBytesWritten == D_LEN * 4) - writer.close } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray @@ -201,7 +210,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft test("copyStreamsWithNIO with a channel") { (0 until NUM_PARTITIONS).foreach{ p => val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.toChannel() + val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() intBuffer.put(data(p)) @@ -209,10 +218,9 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft out.write(byteBuffer.array()) out.close() val in = new FileInputStream(tempFile) - Utils.copyFileStreamNIO(in.getChannel, channel, 0, D_LEN * 4) - in.close() + channel.transferFrom(in.getChannel, 0L, byteBuffer.remaining()) + channel.close() assert(writer.getNumBytesWritten == D_LEN * 4) - writer.close } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray From 7b44ed2d64ffb92e3be35e3abe735fcf66b7a80a Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 26 Jun 2019 18:06:31 -0700 Subject: [PATCH 09/32] Remove shuffle location support. --- .../api/shuffle/MapShuffleLocations.java | 39 ------ .../spark/api/shuffle/ShuffleLocation.java | 25 ---- .../api/shuffle/ShuffleMapOutputWriter.java | 2 +- .../sort/BypassMergeSortShuffleWriter.java | 7 +- .../sort/DefaultMapShuffleLocations.java | 76 ----------- .../shuffle/sort/UnsafeShuffleWriter.java | 13 +- .../io/DefaultShuffleExecutorComponents.java | 2 +- .../io/DefaultShuffleMapOutputWriter.java | 8 +- .../sort/io/DefaultShuffleWriteSupport.java | 9 +- .../org/apache/spark/MapOutputTracker.scala | 29 ++-- .../apache/spark/scheduler/MapStatus.scala | 125 +++--------------- .../spark/serializer/KryoSerializer.scala | 6 +- .../shuffle/BlockStoreShuffleReader.scala | 12 +- .../shuffle/sort/SortShuffleWriter.scala | 7 +- .../sort/UnsafeShuffleWriterSuite.java | 4 +- .../test/org/apache/spark/JavaAPISuite.java | 4 +- .../apache/spark/MapOutputTrackerSuite.scala | 39 +++--- .../scala/org/apache/spark/ShuffleSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 81 +++++------- .../spark/scheduler/MapStatusSuite.scala | 16 +-- .../scheduler/SchedulerIntegrationSuite.scala | 2 +- .../serializer/KryoSerializerSuite.scala | 3 +- .../BlockStoreShuffleReaderSuite.scala | 7 +- .../BypassMergeSortShuffleWriterSuite.scala | 3 +- .../shuffle/sort/SortShuffleWriterSuite.scala | 19 ++- .../DefaultShuffleMapOutputWriterSuite.scala | 1 - 26 files changed, 127 insertions(+), 418 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java delete mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java delete mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java diff --git a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java b/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java deleted file mode 100644 index b0aed4d08d38..000000000000 --- a/core/src/main/java/org/apache/spark/api/shuffle/MapShuffleLocations.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * 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.api.shuffle; - -import org.apache.spark.annotation.Experimental; - -import java.io.Serializable; - -/** - * Represents metadata about where shuffle blocks were written in a single map task. - *

- * This is optionally returned by shuffle writers. The inner shuffle locations may - * be accessed by shuffle readers. Shuffle locations are only necessary when the - * location of shuffle blocks needs to be managed by the driver; shuffle plugins - * may choose to use an external database or other metadata management systems to - * track the locations of shuffle blocks instead. - */ -@Experimental -public interface MapShuffleLocations extends Serializable { - - /** - * Get the location for a given shuffle block written by this map task. - */ - ShuffleLocation getLocationForBlock(int reduceId); -} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java deleted file mode 100644 index 87eb497098e0..000000000000 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleLocation.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * 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.api.shuffle; - -/** - * Marker interface representing a location of a shuffle block. Implementations of shuffle readers - * and writers are expected to cast this down to an implementation-specific representation. - */ -public interface ShuffleLocation { -} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java index 062cf4ff0fba..6822ed05b911 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java @@ -32,7 +32,7 @@ public interface ShuffleMapOutputWriter { ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException; - Optional commitAllPartitions() throws IOException; + void commitAllPartitions() throws IOException; void abort(Throwable error) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 128b90429209..85270f273a89 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -40,7 +40,6 @@ 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; @@ -134,10 +133,9 @@ public void write(Iterator> records) throws IOException { try { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; - Optional blockLocs = mapOutputWriter.commitAllPartitions(); + mapOutputWriter.commitAllPartitions(); mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), - blockLocs.orNull(), partitionLengths); return; } @@ -171,10 +169,9 @@ public void write(Iterator> records) throws IOException { } partitionLengths = writePartitionedData(mapOutputWriter); - Optional mapLocations = mapOutputWriter.commitAllPartitions(); + mapOutputWriter.commitAllPartitions(); mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), - mapLocations.orNull(), partitionLengths); } catch (Exception e) { try { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java deleted file mode 100644 index ffd97c0f2660..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultMapShuffleLocations.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * 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.sort; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; - -import org.apache.spark.api.shuffle.MapShuffleLocations; -import org.apache.spark.api.shuffle.ShuffleLocation; -import org.apache.spark.storage.BlockManagerId; - -import java.util.Objects; - -public class DefaultMapShuffleLocations implements MapShuffleLocations, ShuffleLocation { - - /** - * We borrow the cache size from the BlockManagerId's cache - around 1MB, which should be - * feasible. - */ - private static final LoadingCache - DEFAULT_SHUFFLE_LOCATIONS_CACHE = - CacheBuilder.newBuilder() - .maximumSize(BlockManagerId.blockManagerIdCacheSize()) - .build(new CacheLoader() { - @Override - public DefaultMapShuffleLocations load(BlockManagerId blockManagerId) { - return new DefaultMapShuffleLocations(blockManagerId); - } - }); - - private final BlockManagerId location; - - public DefaultMapShuffleLocations(BlockManagerId blockManagerId) { - this.location = blockManagerId; - } - - public static DefaultMapShuffleLocations get(BlockManagerId blockManagerId) { - return DEFAULT_SHUFFLE_LOCATIONS_CACHE.getUnchecked(blockManagerId); - } - - @Override - public ShuffleLocation getLocationForBlock(int reduceId) { - return this; - } - - public BlockManagerId getBlockManagerId() { - return location; - } - - @Override - public boolean equals(Object other) { - return other instanceof DefaultMapShuffleLocations - && Objects.equals(((DefaultMapShuffleLocations) other).location, location); - } - - @Override - public int hashCode() { - return Objects.hashCode(location); - } -} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index a30345a93139..ac3ff1ec71db 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -38,7 +38,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; @@ -219,7 +218,6 @@ void closeAndWriteOutput() throws IOException { final ShuffleMapOutputWriter mapWriter = shuffleWriteSupport .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); final long[] partitionLengths; - Optional mapLocations; try { try { partitionLengths = mergeSpills(spills, mapWriter); @@ -230,7 +228,7 @@ void closeAndWriteOutput() throws IOException { } } } - mapLocations = mapWriter.commitAllPartitions(); + mapWriter.commitAllPartitions(); } catch (Exception e) { try { mapWriter.abort(e); @@ -241,7 +239,6 @@ void closeAndWriteOutput() throws IOException { } mapStatus = MapStatus$.MODULE$.apply( blockManager.shuffleServerId(), - mapLocations.orNull(), partitionLengths); } @@ -434,18 +431,10 @@ private long[] mergeSpillsWithTransferTo( ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); TransferrableWritableByteChannel partitionChannel = null; try { -<<<<<<< HEAD - writer = mapWriter.getPartitionWriter(partition); - WritableByteChannel channel = writer.toChannel(); -||||||| parent of ca4448ec6d... [SPARK-25299] Propose a new NIO transfer API for partition writing. (#535) - writer = mapWriter.getNextPartitionWriter(); - WritableByteChannel channel = writer.toChannel(); -======= partitionChannel = writer instanceof SupportsTransferTo ? ((SupportsTransferTo) writer).openTransferrableChannel() : new DefaultTransferrableWritableByteChannel( Channels.newChannel(writer.openStream())); ->>>>>>> ca4448ec6d... [SPARK-25299] Propose a new NIO transfer API for partition writing. (#535) for (int i = 0; i < spills.length; i++) { long partitionLengthInSpill = 0L; partitionLengthInSpill += spills[i].partitionLengths[partition]; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java index f7ec202ef4b9..76e87a674025 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java @@ -46,6 +46,6 @@ public ShuffleWriteSupport writes() { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } - return new DefaultShuffleWriteSupport(sparkConf, blockResolver, blockManager.shuffleServerId()); + return new DefaultShuffleWriteSupport(sparkConf, blockResolver); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index e83db4e4bcef..cf57ba6f9cfd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -29,13 +29,11 @@ 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; @@ -56,7 +54,6 @@ public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { private final int bufferSize; private int lastPartitionId = -1; private long currChannelPosition; - private final BlockManagerId shuffleServerId; private final File outputFile; private File outputTempFile; @@ -69,13 +66,11 @@ public DefaultShuffleMapOutputWriter( int shuffleId, int mapId, int numPartitions, - BlockManagerId shuffleServerId, ShuffleWriteMetricsReporter metrics, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; this.mapId = mapId; - this.shuffleServerId = shuffleServerId; this.metrics = metrics; this.blockResolver = blockResolver; this.bufferSize = @@ -104,11 +99,10 @@ public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOExcep } @Override - public Optional commitAllPartitions() throws IOException { + public void commitAllPartitions() throws IOException { cleanUp(); File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); - return Optional.of(DefaultMapShuffleLocations.get(shuffleServerId)); } @Override diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java index 86f158349568..ff0b6dc370f7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java @@ -28,15 +28,12 @@ public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { private final SparkConf sparkConf; private final IndexShuffleBlockResolver blockResolver; - private final BlockManagerId shuffleServerId; public DefaultShuffleWriteSupport( SparkConf sparkConf, - IndexShuffleBlockResolver blockResolver, - BlockManagerId shuffleServerId) { + IndexShuffleBlockResolver blockResolver) { this.sparkConf = sparkConf; this.blockResolver = blockResolver; - this.shuffleServerId = shuffleServerId; } @Override @@ -45,7 +42,7 @@ public ShuffleMapOutputWriter createMapOutputWriter( int mapId, int numPartitions) { return new DefaultShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, shuffleServerId, - TaskContext.get().taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); + shuffleId, mapId, numPartitions, + TaskContext.get().taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 81a8001fde68..5c820f5a2ed0 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -28,7 +28,6 @@ import scala.concurrent.duration.Duration import scala.reflect.ClassTag import scala.util.control.NonFatal -import org.apache.spark.api.shuffle.{MapShuffleLocations, ShuffleLocation} import org.apache.spark.broadcast.{Broadcast, BroadcastManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ @@ -282,9 +281,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging } // For testing - def getMapSizesByShuffleLocation(shuffleId: Int, reduceId: Int) - : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { - getMapSizesByShuffleLocation(shuffleId, reduceId, reduceId + 1) + def getMapSizesByExecutorId(shuffleId: Int, reduceId: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { + getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1) } /** @@ -296,8 +295,8 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * and the second item is a sequence of (shuffle block id, shuffle block size) tuples * describing the shuffle blocks that are stored at that block manager. */ - def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] + def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] /** * Deletes map output status information for the specified shuffle stage. @@ -646,8 +645,8 @@ private[spark] class MapOutputTrackerMaster( // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. // This method is only called in local-mode. - def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { + def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") shuffleStatuses.get(shuffleId) match { case Some (shuffleStatus) => @@ -683,13 +682,12 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr private val fetching = new HashSet[Int] // Get blocks sizes by executor Id. Note that zero-sized blocks are excluded in the result. - override def getMapSizesByShuffleLocation(shuffleId: Int, startPartition: Int, endPartition: Int) - : Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { + override def getMapSizesByExecutorId(shuffleId: Int, startPartition: Int, endPartition: Int) + : Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { logDebug(s"Fetching outputs for shuffle $shuffleId, partitions $startPartition-$endPartition") val statuses = getStatuses(shuffleId) try { - MapOutputTracker.convertMapStatuses( - shuffleId, startPartition, endPartition, statuses) + MapOutputTracker.convertMapStatuses(shuffleId, startPartition, endPartition, statuses) } catch { case e: MetadataFetchFailedException => // We experienced a fetch failure so our mapStatuses cache is outdated; clear it: @@ -873,9 +871,9 @@ private[spark] object MapOutputTracker extends Logging { shuffleId: Int, startPartition: Int, endPartition: Int, - statuses: Array[MapStatus]): Iterator[(ShuffleLocation, Seq[(BlockId, Long)])] = { + statuses: Array[MapStatus]): Iterator[(BlockManagerId, Seq[(BlockId, Long)])] = { assert (statuses != null) - val splitsByAddress = new HashMap[ShuffleLocation, ListBuffer[(BlockId, Long)]] + val splitsByAddress = new HashMap[BlockManagerId, ListBuffer[(BlockId, Long)]] for ((status, mapId) <- statuses.iterator.zipWithIndex) { if (status == null) { val errorMessage = s"Missing an output location for shuffle $shuffleId" @@ -885,8 +883,7 @@ private[spark] object MapOutputTracker extends Logging { for (part <- startPartition until endPartition) { val size = status.getSizeForBlock(part) if (size != 0) { - val shuffleLoc = status.mapShuffleLocations.getLocationForBlock(part) - splitsByAddress.getOrElseUpdate(shuffleLoc, ListBuffer()) += + splitsByAddress.getOrElseUpdate(status.location, ListBuffer()) += ((ShuffleBlockId(shuffleId, mapId, part), size)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index a61f9bd14ef2..64f0a060a247 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -24,9 +24,7 @@ import scala.collection.mutable import org.roaringbitmap.RoaringBitmap import org.apache.spark.SparkEnv -import org.apache.spark.api.shuffle.MapShuffleLocations import org.apache.spark.internal.config -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.Utils @@ -35,16 +33,7 @@ import org.apache.spark.util.Utils * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. */ private[spark] sealed trait MapStatus { - - /** - * Locations where this task stored shuffle blocks. - * - * May be null if the MapOutputTracker is not tracking the location of shuffle blocks, leaving it - * up to the implementation of shuffle plugins to do so. - */ - def mapShuffleLocations: MapShuffleLocations - - /** Location where the task was run. */ + /** Location where this task was run. */ def location: BlockManagerId /** @@ -67,31 +56,11 @@ private[spark] object MapStatus { .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get) - // A temporary concession to the fact that we only expect implementations of shuffle provided by - // Spark to be storing shuffle locations in the driver, meaning we want to introduce as little - // serialization overhead as possible in such default cases. - // - // If more similar cases arise, consider adding a serialization API for these shuffle locations. - private val DEFAULT_MAP_SHUFFLE_LOCATIONS_ID: Byte = 0 - private val NON_DEFAULT_MAP_SHUFFLE_LOCATIONS_ID: Byte = 1 - - /** - * Visible for testing. - */ def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { - apply(loc, DefaultMapShuffleLocations.get(loc), uncompressedSizes) - } - - def apply( - loc: BlockManagerId, - mapShuffleLocs: MapShuffleLocations, - uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { - HighlyCompressedMapStatus( - loc, mapShuffleLocs, uncompressedSizes) + HighlyCompressedMapStatus(loc, uncompressedSizes) } else { - new CompressedMapStatus( - loc, mapShuffleLocs, uncompressedSizes) + new CompressedMapStatus(loc, uncompressedSizes) } } @@ -122,89 +91,41 @@ private[spark] object MapStatus { math.pow(LOG_BASE, compressedSize & 0xFF).toLong } } - - def writeLocations( - loc: BlockManagerId, - mapShuffleLocs: MapShuffleLocations, - out: ObjectOutput): Unit = { - if (mapShuffleLocs != null) { - out.writeBoolean(true) - if (mapShuffleLocs.isInstanceOf[DefaultMapShuffleLocations] - && mapShuffleLocs.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId == loc) { - out.writeByte(MapStatus.DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) - } else { - out.writeByte(MapStatus.NON_DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) - out.writeObject(mapShuffleLocs) - } - } else { - out.writeBoolean(false) - } - loc.writeExternal(out) - } - - def readLocations(in: ObjectInput): (BlockManagerId, MapShuffleLocations) = { - if (in.readBoolean()) { - val locId = in.readByte() - if (locId == MapStatus.DEFAULT_MAP_SHUFFLE_LOCATIONS_ID) { - val blockManagerId = BlockManagerId(in) - (blockManagerId, DefaultMapShuffleLocations.get(blockManagerId)) - } else { - val mapShuffleLocations = in.readObject().asInstanceOf[MapShuffleLocations] - val blockManagerId = BlockManagerId(in) - (blockManagerId, mapShuffleLocations) - } - } else { - val blockManagerId = BlockManagerId(in) - (blockManagerId, null) - } - } } + /** * A [[MapStatus]] implementation that tracks the size of each block. Size for each block is * represented using a single byte. * - * @param loc Location were the task is being executed. - * @param mapShuffleLocs locations where the task stored its shuffle blocks - may be null. + * @param loc location where the task is being executed. * @param compressedSizes size of the blocks, indexed by reduce partition id. */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var mapShuffleLocs: MapShuffleLocations, private[this] var compressedSizes: Array[Byte]) extends MapStatus with Externalizable { - // For deserialization only - protected def this() = this(null, null, null.asInstanceOf[Array[Byte]]) + protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only - def this( - loc: BlockManagerId, - mapShuffleLocations: MapShuffleLocations, - uncompressedSizes: Array[Long]) { - this( - loc, - mapShuffleLocations, - uncompressedSizes.map(MapStatus.compressSize)) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.map(MapStatus.compressSize)) } override def location: BlockManagerId = loc - override def mapShuffleLocations: MapShuffleLocations = mapShuffleLocs - override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - MapStatus.writeLocations(loc, mapShuffleLocs, out) + loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val (deserializedLoc, deserializedMapShuffleLocs) = MapStatus.readLocations(in) - loc = deserializedLoc - mapShuffleLocs = deserializedMapShuffleLocs + loc = BlockManagerId(in) val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) @@ -217,7 +138,6 @@ private[spark] class CompressedMapStatus( * plus a bitmap for tracking which blocks are empty. * * @param loc location where the task is being executed - * @param mapShuffleLocs location where the task stored shuffle blocks - may be null * @param numNonEmptyBlocks the number of non-empty blocks * @param emptyBlocks a bitmap tracking which blocks are empty * @param avgSize average size of the non-empty and non-huge blocks @@ -225,7 +145,6 @@ private[spark] class CompressedMapStatus( */ private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, - private[this] var mapShuffleLocs: MapShuffleLocations, private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, @@ -236,12 +155,10 @@ private[spark] class HighlyCompressedMapStatus private ( require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, null, -1, null, -1, null) // For deserialization only + protected def this() = this(null, -1, null, -1, null) // For deserialization only override def location: BlockManagerId = loc - override def mapShuffleLocations: MapShuffleLocations = mapShuffleLocs - override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -255,7 +172,7 @@ private[spark] class HighlyCompressedMapStatus private ( } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - MapStatus.writeLocations(loc, mapShuffleLocs, out) + loc.writeExternal(out) emptyBlocks.writeExternal(out) out.writeLong(avgSize) out.writeInt(hugeBlockSizes.size) @@ -266,9 +183,7 @@ private[spark] class HighlyCompressedMapStatus private ( } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val (deserializedLoc, deserializedMapShuffleLocs) = MapStatus.readLocations(in) - loc = deserializedLoc - mapShuffleLocs = deserializedMapShuffleLocs + loc = BlockManagerId(in) emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() @@ -284,10 +199,7 @@ private[spark] class HighlyCompressedMapStatus private ( } private[spark] object HighlyCompressedMapStatus { - def apply( - loc: BlockManagerId, - mapShuffleLocs: MapShuffleLocations, - uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -327,12 +239,7 @@ private[spark] object HighlyCompressedMapStatus { } emptyBlocks.trim() emptyBlocks.runOptimize() - new HighlyCompressedMapStatus( - loc, - mapShuffleLocs, - numNonEmptyBlocks, - emptyBlocks, - avgSize, - hugeBlockSizes) + new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, + hugeBlockSizes) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 48923666bab6..20774c8d999c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,7 +31,7 @@ import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSe import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} import com.esotericsoftware.kryo.pool.{KryoCallback, KryoFactory, KryoPool} -import com.esotericsoftware.kryo.serializers.{ExternalizableSerializer, JavaSerializer => KryoJavaSerializer} +import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} import org.roaringbitmap.RoaringBitmap @@ -151,8 +151,6 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[SerializableConfiguration], new KryoJavaSerializer()) kryo.register(classOf[SerializableJobConf], new KryoJavaSerializer()) kryo.register(classOf[PythonBroadcast], new KryoJavaSerializer()) - kryo.register(classOf[CompressedMapStatus], new ExternalizableSerializer()) - kryo.register(classOf[HighlyCompressedMapStatus], new ExternalizableSerializer()) kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas)) kryo.register(classOf[GenericData.Record], new GenericAvroSerializer(avroSchemas)) @@ -456,6 +454,8 @@ private[serializer] object KryoSerializer { private val toRegister: Seq[Class[_]] = Seq( ByteBuffer.allocate(1).getClass, classOf[StorageLevel], + classOf[CompressedMapStatus], + classOf[HighlyCompressedMapStatus], classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Boolean]], diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 4d559556360c..c7843710413d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -20,8 +20,7 @@ package org.apache.spark.shuffle import org.apache.spark._ import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer.SerializerManager -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations -import org.apache.spark.storage.{BlockId, BlockManager, ShuffleBlockFetcherIterator} +import org.apache.spark.storage.{BlockManager, ShuffleBlockFetcherIterator} import org.apache.spark.util.CompletionIterator import org.apache.spark.util.collection.ExternalSorter @@ -48,14 +47,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( context, blockManager.shuffleClient, blockManager, - mapOutputTracker.getMapSizesByShuffleLocation(handle.shuffleId, startPartition, endPartition) - .map { - case (loc: DefaultMapShuffleLocations, blocks: Seq[(BlockId, Long)]) => - (loc.getBlockManagerId, blocks) - case _ => - throw new UnsupportedOperationException("Not allowed to using non-default map shuffle" + - " locations yet.") - }, + mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 1fcae684b005..62316f384b64 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,11 +67,8 @@ private[spark] class SortShuffleWriter[K, V, C]( val mapOutputWriter = writeSupport.createMapOutputWriter( dep.shuffleId, mapId, dep.partitioner.numPartitions) val partitionLengths = sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - val mapLocations = mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus( - blockManager.shuffleServerId, - mapLocations.orNull(), - partitionLengths) + mapOutputWriter.commitAllPartitions() + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 3c172a027ca0..64227e4aefbb 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -189,7 +189,7 @@ private UnsafeShuffleWriter createWriter( taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver, blockManager.shuffleServerId())); + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); } private void assertSpillFilesWereCleanedUp() { @@ -549,7 +549,7 @@ public void testPeakMemoryUsed() throws Exception { taskContext, conf, taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver, blockManager.shuffleServerId())); + new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index a8252e03b5c1..08f0d65abff1 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -94,7 +94,9 @@ public void setUp() { @After public void tearDown() { - sc.stop(); + if (sc != null) { + sc.stop(); + } sc = null; } diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 0a77c4f6d583..d86975964b55 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -29,7 +29,6 @@ import org.apache.spark.internal.config.Network.{RPC_ASK_TIMEOUT, RPC_MESSAGE_MA import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockManagerId, ShuffleBlockId} class MapOutputTrackerSuite extends SparkFunSuite { @@ -68,13 +67,10 @@ class MapOutputTrackerSuite extends SparkFunSuite { Array(1000L, 10000L))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(10000L, 1000L))) - val statuses = tracker.getMapSizesByShuffleLocation(10, 0) + val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === - Seq( - (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), - ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), - (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)), - ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), + (BlockManagerId("b", "hostB", 1000), ArrayBuffer((ShuffleBlockId(10, 1, 0), size10000)))) .toSet) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.stop() @@ -94,11 +90,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByShuffleLocation(10, 0).nonEmpty) + assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) assert(0 == tracker.getNumCachedSerializedBroadcast) tracker.unregisterShuffle(10) assert(!tracker.containsShuffle(10)) - assert(tracker.getMapSizesByShuffleLocation(10, 0).isEmpty) + assert(tracker.getMapSizesByExecutorId(10, 0).isEmpty) tracker.stop() rpcEnv.shutdown() @@ -125,7 +121,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // The remaining reduce task might try to grab the output despite the shuffle failure; // this should cause it to fail, and the scheduler will ignore the failure due to the // stage already being aborted. - intercept[FetchFailedException] { tracker.getMapSizesByShuffleLocation(10, 1) } + intercept[FetchFailedException] { tracker.getMapSizesByExecutorId(10, 1) } tracker.stop() rpcEnv.shutdown() @@ -147,26 +143,24 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.registerShuffle(10, 1) slaveTracker.updateEpoch(masterTracker.getEpoch) // This is expected to fail because no outputs have been registered for the shuffle. - intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("a", "hostA", 1000), Array(1000L))) slaveTracker.updateEpoch(masterTracker.getEpoch) - assert(slaveTracker.getMapSizesByShuffleLocation(10, 0).toSeq === - Seq( - (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), - ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) + assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) assert(0 == masterTracker.getNumCachedSerializedBroadcast) val masterTrackerEpochBeforeLossOfMapOutput = masterTracker.getEpoch masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000)) assert(masterTracker.getEpoch > masterTrackerEpochBeforeLossOfMapOutput) slaveTracker.updateEpoch(masterTracker.getEpoch) - intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } // failure should be cached - intercept[FetchFailedException] { slaveTracker.getMapSizesByShuffleLocation(10, 0) } + intercept[FetchFailedException] { slaveTracker.getMapSizesByExecutorId(10, 0) } assert(0 == masterTracker.getNumCachedSerializedBroadcast) masterTracker.stop() @@ -267,11 +261,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { // being sent. masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => - val bmId = BlockManagerId("999", "mps", 1000) masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - bmId, - DefaultMapShuffleLocations.get(bmId), - Array.fill[Long](4000000)(0))) + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -324,11 +315,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(size10000, size0, size1000, size0))) assert(tracker.containsShuffle(10)) - assert(tracker.getMapSizesByShuffleLocation(10, 0, 4).toSeq === + assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === Seq( - (DefaultMapShuffleLocations.get(BlockManagerId("a", "hostA", 1000)), + (BlockManagerId("a", "hostA", 1000), Seq((ShuffleBlockId(10, 0, 1), size1000), (ShuffleBlockId(10, 0, 3), size10000))), - (DefaultMapShuffleLocations.get(BlockManagerId("b", "hostB", 1000)), + (BlockManagerId("b", "hostB", 1000), Seq((ShuffleBlockId(10, 1, 0), size10000), (ShuffleBlockId(10, 1, 2), size1000))) ) ) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 83026c002f1b..90c790cefcca 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -73,7 +73,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // All blocks must have non-zero size (0 until NUM_BLOCKS).foreach { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) assert(statuses.forall(_._2.forall(blockIdSizePair => blockIdSizePair._2 > 0))) } } @@ -112,7 +112,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) statuses.flatMap(_._2.map(_._2)) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) @@ -137,7 +137,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(c.count === 4) val blockSizes = (0 until NUM_BLOCKS).flatMap { id => - val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, id) + val statuses = SparkEnv.get.mapOutputTracker.getMapSizesByExecutorId(shuffleId, id) statuses.flatMap(_._2.map(_._2)) } val nonEmptyBlocks = blockSizes.filter(x => x > 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 20b6dcd26966..d58ee4e651e1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -29,14 +29,12 @@ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.api.shuffle.MapShuffleLocations import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.internal.config import org.apache.spark.rdd.{DeterministicLevel, RDD} import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.shuffle.{FetchFailedException, MetadataFetchFailedException} -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, CallSite, LongAccumulator, Utils} @@ -715,8 +713,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), (Success, makeMapStatus("hostB", 1)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -742,10 +740,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", reduceRdd.partitions.length)))) // we can see both result blocks now - assert(mapOutputTracker - .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) - .toSet === HashSet("hostA", "hostB")) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === + HashSet("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) assertDataStructuresEmpty() @@ -783,11 +779,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi runEvent(ExecutorLost("exec-hostA", event)) if (expectFileLoss) { intercept[MetadataFetchFailedException] { - mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0) + mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) } } else { - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) } } } @@ -1080,10 +1076,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) // The MapOutputTracker should know about both map output locations. - assert(mapOutputTracker - .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) - .toSet === HashSet("hostA", "hostB")) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === + HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( @@ -1212,14 +1206,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi (Success, makeMapStatus("hostA", 2)), (Success, makeMapStatus("hostB", 2)))) // The MapOutputTracker should know about both map output locations. - assert(mapOutputTracker - .getMapSizesByShuffleLocation(shuffleId, 0) - .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) - .toSet === HashSet("hostA", "hostB")) - assert(mapOutputTracker - .getMapSizesByShuffleLocation(shuffleId, 1) - .map(_._1.asInstanceOf[DefaultMapShuffleLocations].getBlockManagerId.host) - .toSet === HashSet("hostA", "hostB")) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === + HashSet("hostA", "hostB")) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 1).map(_._1.host).toSet === + HashSet("hostA", "hostB")) // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(makeCompletionEvent( @@ -1409,8 +1399,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi Success, makeMapStatus("hostA", reduceRdd.partitions.size))) assert(shuffleStage.numAvailableOutputs === 2) - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostB"), makeShuffleLocation("hostA"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) // finish the next stage normally, which completes the job complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -1564,7 +1554,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi reduceIdx <- reduceIdxs } { // this would throw an exception if the map status hadn't been registered - val statuses = mapOutputTracker.getMapSizesByShuffleLocation(stage, reduceIdx) + val statuses = mapOutputTracker.getMapSizesByExecutorId(stage, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) @@ -1616,7 +1606,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // check that we have all the map output for stage 0 (0 until reduceRdd.partitions.length).foreach { reduceIdx => - val statuses = mapOutputTracker.getMapSizesByShuffleLocation(0, reduceIdx) + val statuses = mapOutputTracker.getMapSizesByExecutorId(0, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) @@ -1815,8 +1805,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // have hostC complete the resubmitted task complete(taskSets(1), Seq((Success, makeMapStatus("hostC", 1)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) // Make sure that the reduce stage was now submitted. assert(taskSets.size === 3) @@ -2078,8 +2068,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"))) // Reducer should run on the same host that map task ran val reduceTaskSet = taskSets(1) @@ -2124,8 +2114,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"))) + assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"))) // Reducer should run where RDD 2 has preferences, even though it also has a shuffle dep val reduceTaskSet = taskSets(1) @@ -2288,8 +2278,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) // When attempting the second stage, show a fetch failure @@ -2304,9 +2294,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(2).stageId === 0) complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) - + assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) assert(listener2.results.size === 0) // Second stage listener should still not have a result // Stage 1 should now be running as task set 3; make its first task succeed @@ -2314,8 +2303,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(3), Seq( (Success, makeMapStatus("hostB", rdd2.partitions.length)), (Success, makeMapStatus("hostD", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(dep2.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostB"), makeShuffleLocation("hostD"))) + assert(mapOutputTracker.getMapSizesByExecutorId(dep2.shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostB"), makeBlockManagerId("hostD"))) assert(listener2.results.size === 1) // Finally, the reduce job should be running as task set 4; make it see a fetch failure, @@ -2353,8 +2342,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", rdd1.partitions.length)), (Success, makeMapStatus("hostB", rdd1.partitions.length)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - HashSet(makeShuffleLocation("hostA"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === + HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) // When attempting stage1, trigger a fetch failure. @@ -2379,8 +2368,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(2).stageId === 0) complete(taskSets(2), Seq( (Success, makeMapStatus("hostC", rdd2.partitions.length)))) - assert(mapOutputTracker.getMapSizesByShuffleLocation(dep1.shuffleId, 0).map(_._1).toSet === - Set(makeShuffleLocation("hostC"), makeShuffleLocation("hostB"))) + assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === + Set(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) // After stage0 is finished, stage1 will be submitted and found there is no missing // partitions in it. Then listener got triggered. @@ -2993,10 +2982,6 @@ object DAGSchedulerSuite { def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) - - def makeShuffleLocation(host: String): MapShuffleLocations = { - DefaultMapShuffleLocations.get(makeBlockManagerId(host)) - } } object FailThisAttempt { diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 3c786c0927bc..d7304ce714c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -28,7 +28,6 @@ import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.LocalSparkContext._ import org.apache.spark.internal.config import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId class MapStatusSuite extends SparkFunSuite { @@ -65,7 +64,6 @@ class MapStatusSuite extends SparkFunSuite { val bmId = BlockManagerId("a", "b", 10) val status = MapStatus( bmId, - DefaultMapShuffleLocations.get(bmId), sizes) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { @@ -80,7 +78,7 @@ class MapStatusSuite extends SparkFunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, null, sizes) + val status = MapStatus(null, sizes) assert(status.isInstanceOf[HighlyCompressedMapStatus]) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) @@ -92,12 +90,10 @@ class MapStatusSuite extends SparkFunSuite { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) val bmId = BlockManagerId("a", "b", 10) - val loc = DefaultMapShuffleLocations.get(bmId) - val status = MapStatus(bmId, loc, sizes) + val status = MapStatus(bmId, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) - assert(status1.location == loc.getBlockManagerId) - assert(status1.mapShuffleLocations == loc) + assert(status1.location == bmId) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -116,12 +112,10 @@ class MapStatusSuite extends SparkFunSuite { val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length val bmId = BlockManagerId("a", "b", 10) - val loc = DefaultMapShuffleLocations.get(bmId) - val status = MapStatus(bmId, loc, sizes) + val status = MapStatus(bmId, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location === bmId) - assert(status1.mapShuffleLocations === loc) for (i <- 0 until threshold) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -175,7 +169,7 @@ class MapStatusSuite extends SparkFunSuite { // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray val bmId = BlockManagerId("exec-0", "host-0", 100) - val status1 = MapStatus(bmId, DefaultMapShuffleLocations.get(bmId), sizes) + val status1 = MapStatus(bmId, sizes) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index ec828db9391d..7a1f5b690ab8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -191,7 +191,7 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa shuffleId <- shuffleIds reduceIdx <- (0 until nParts) } { - val statuses = taskScheduler.mapOutputTracker.getMapSizesByShuffleLocation( + val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId( shuffleId, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 63f6942dee18..8740ca09515d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -36,7 +36,6 @@ import org.apache.spark.internal.config._ import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.ThreadUtils @@ -354,7 +353,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val bmId = BlockManagerId("exec-1", "host", 1234) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => ser.serialize(HighlyCompressedMapStatus( - bmId, DefaultMapShuffleLocations.get(bmId), blockSizes)) + bmId, blockSizes)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index b3073addb7cc..1aa699e7ad6a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark._ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} -import org.apache.spark.shuffle.sort.DefaultMapShuffleLocations import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} /** @@ -103,7 +102,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Make a mocked MapOutputTracker for the shuffle reader to use to determine what // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) - when(mapOutputTracker.getMapSizesByShuffleLocation( + when(mapOutputTracker.getMapSizesByExecutorId( shuffleId, reduceId, reduceId + 1)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. @@ -111,9 +110,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext val shuffleBlockId = ShuffleBlockId(shuffleId, mapId, reduceId) (shuffleBlockId, byteOutputStream.size().toLong) } - Seq( - (DefaultMapShuffleLocations.get(localBlockManagerId), shuffleBlockIdsAndSizes)) - .toIterator + Seq((localBlockManagerId, shuffleBlockIdsAndSizes)).toIterator } // Create a mocked shuffle handle to pass into HashShuffleReader. diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index c491f39249c8..b16bf5194eeb 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -129,8 +129,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte metricsSystem = null, taskMetrics = taskMetrics)) - writeSupport = new DefaultShuffleWriteSupport( - conf, blockResolver, BlockManagerId("0", "localhost", 7090)) + writeSupport = new DefaultShuffleWriteSupport(conf, blockResolver) } override def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index 690bcd990525..e99e179821c5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -21,10 +21,12 @@ import org.mockito.Mockito._ import org.mockito.MockitoAnnotations import org.scalatest.Matchers -import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite, TaskContext} +import org.apache.spark.api.shuffle.ShuffleWriteSupport import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} +import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport import org.apache.spark.util.Utils @@ -35,9 +37,11 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with private var shuffleHandle: BaseShuffleHandle[Int, Int, Int] = _ private val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) private val serializer = new JavaSerializer(conf) + private var writeSupport: ShuffleWriteSupport = _ override def beforeEach(): Unit = { super.beforeEach() + writeSupport = new DefaultShuffleWriteSupport(conf, shuffleBlockResolver) MockitoAnnotations.initMocks(this) val partitioner = new Partitioner() { def numPartitions = numMaps @@ -53,6 +57,11 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } } + override def afterEach(): Unit = { + super.afterEach() + TaskContext.unset() + } + override def afterAll(): Unit = { try { shuffleBlockResolver.stop() @@ -63,11 +72,13 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with test("write empty iterator") { val context = MemoryTestingUtils.fakeTaskContext(sc.env) + TaskContext.setTaskContext(context) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, mapId = 1, - context) + context, + writeSupport) writer.write(Iterator.empty) writer.stop(success = true) val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 1) @@ -79,12 +90,14 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with test("write with some records") { val context = MemoryTestingUtils.fakeTaskContext(sc.env) + TaskContext.setTaskContext(context) val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, mapId = 2, - context) + context, + writeSupport) writer.write(records.toIterator) writer.stop(success = true) val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 2) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala index 1f4ef0f20399..e200eb230468 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala @@ -94,7 +94,6 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft 0, 0, NUM_PARTITIONS, - BlockManagerId("0", "localhost", 9099), shuffleWriteMetrics, blockResolver, conf) From df75f1f5984f596ec63fb5907b9250c214dcdf63 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 26 Jun 2019 18:20:35 -0700 Subject: [PATCH 10/32] Remove changes to UnsafeShuffleWriter --- .../shuffle/sort/UnsafeShuffleWriter.java | 229 ++++++++++-------- .../shuffle/sort/SortShuffleManager.scala | 4 +- .../sort/UnsafeShuffleWriterSuite.java | 52 ++-- 3 files changed, 144 insertions(+), 141 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index ac3ff1ec71db..9d05f03613ce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -17,7 +17,6 @@ package org.apache.spark.shuffle.sort; -import java.nio.channels.Channels; import javax.annotation.Nullable; import java.io.*; import java.nio.channels.FileChannel; @@ -32,21 +31,18 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.io.ByteStreams; import com.google.common.io.Closeables; +import com.google.common.io.Files; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.*; import org.apache.spark.annotation.Private; -import org.apache.spark.api.java.Optional; -import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; -import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; -import org.apache.spark.api.shuffle.ShufflePartitionWriter; -import org.apache.spark.api.shuffle.ShuffleWriteSupport; -import org.apache.spark.api.shuffle.SupportsTransferTo; import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; +import org.apache.commons.io.output.CloseShieldOutputStream; +import org.apache.commons.io.output.CountingOutputStream; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; @@ -54,9 +50,12 @@ import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; +import org.apache.spark.util.Utils; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -66,14 +65,15 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); @VisibleForTesting + static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; private final BlockManager blockManager; + private final IndexShuffleBlockResolver shuffleBlockResolver; private final TaskMemoryManager memoryManager; private final SerializerInstance serializer; private final Partitioner partitioner; private final ShuffleWriteMetricsReporter writeMetrics; - private final ShuffleWriteSupport shuffleWriteSupport; private final int shuffleId; private final int mapId; private final TaskContext taskContext; @@ -81,6 +81,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final boolean transferToEnabled; private final int initialSortBufferSize; private final int inputBufferSizeInBytes; + private final int outputBufferSizeInBytes; @Nullable private MapStatus mapStatus; @Nullable private ShuffleExternalSorter sorter; @@ -102,15 +103,27 @@ private static final class MyByteArrayOutputStream extends ByteArrayOutputStream */ private boolean stopping = false; + private class CloseAndFlushShieldOutputStream extends CloseShieldOutputStream { + + CloseAndFlushShieldOutputStream(OutputStream outputStream) { + super(outputStream); + } + + @Override + public void flush() { + // do nothing + } + } + public UnsafeShuffleWriter( BlockManager blockManager, + IndexShuffleBlockResolver shuffleBlockResolver, TaskMemoryManager memoryManager, SerializedShuffleHandle handle, int mapId, TaskContext taskContext, SparkConf sparkConf, - ShuffleWriteMetricsReporter writeMetrics, - ShuffleWriteSupport shuffleWriteSupport) throws IOException { + ShuffleWriteMetricsReporter writeMetrics) throws IOException { final int numPartitions = handle.dependency().partitioner().numPartitions(); if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( @@ -119,6 +132,7 @@ public UnsafeShuffleWriter( " reduce partitions"); } this.blockManager = blockManager; + this.shuffleBlockResolver = shuffleBlockResolver; this.memoryManager = memoryManager; this.mapId = mapId; final ShuffleDependency dep = handle.dependency(); @@ -126,7 +140,6 @@ public UnsafeShuffleWriter( this.serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); this.writeMetrics = writeMetrics; - this.shuffleWriteSupport = shuffleWriteSupport; this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); @@ -134,6 +147,8 @@ public UnsafeShuffleWriter( (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; + this.outputBufferSizeInBytes = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE()) * 1024; open(); } @@ -215,31 +230,26 @@ void closeAndWriteOutput() throws IOException { serOutputStream = null; final SpillInfo[] spills = sorter.closeAndGetSpills(); sorter = null; - final ShuffleMapOutputWriter mapWriter = shuffleWriteSupport - .createMapOutputWriter(shuffleId, mapId, partitioner.numPartitions()); final long[] partitionLengths; + final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); + final File tmp = Utils.tempFileWith(output); try { try { - partitionLengths = mergeSpills(spills, mapWriter); + partitionLengths = mergeSpills(spills, tmp); } finally { for (SpillInfo spill : spills) { - if (spill.file.exists() && !spill.file.delete()) { + if (spill.file.exists() && ! spill.file.delete()) { logger.error("Error while deleting spill file {}", spill.file.getPath()); } } } - mapWriter.commitAllPartitions(); - } catch (Exception e) { - try { - mapWriter.abort(e); - } catch (Exception innerE) { - logger.error("Failed to abort the Map Output Writer", innerE); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } - throw e; } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), - partitionLengths); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting @@ -271,8 +281,7 @@ void forceSorterToSpill() throws IOException { * * @return the partition lengths in the merged file. */ - private long[] mergeSpills(SpillInfo[] spills, - ShuffleMapOutputWriter mapWriter) throws IOException { + private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = @@ -280,12 +289,17 @@ private long[] mergeSpills(SpillInfo[] spills, final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); - final int numPartitions = partitioner.numPartitions(); - long[] partitionLengths = new long[numPartitions]; try { if (spills.length == 0) { - return partitionLengths; + new FileOutputStream(outputFile).close(); // Create an empty file + return new long[partitioner.numPartitions()]; + } else if (spills.length == 1) { + // Here, we don't need to perform any metrics updates because the bytes written to this + // output file would have already been counted as shuffle bytes written. + Files.move(spills[0].file, outputFile); + return spills[0].partitionLengths; } else { + final long[] partitionLengths; // There are multiple spills to merge, so none of these spill files' lengths were counted // towards our shuffle write count or shuffle write time. If we use the slow merge path, // then the final output file's size won't necessarily be equal to the sum of the spill @@ -302,14 +316,14 @@ private long[] mergeSpills(SpillInfo[] spills, // that doesn't need to interpret the spilled bytes. if (transferToEnabled && !encryptionEnabled) { logger.debug("Using transferTo-based fast merge"); - partitionLengths = mergeSpillsWithTransferTo(spills, mapWriter); + partitionLengths = mergeSpillsWithTransferTo(spills, outputFile); } else { logger.debug("Using fileStream-based fast merge"); - partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, null); + partitionLengths = mergeSpillsWithFileStream(spills, outputFile, null); } } else { logger.debug("Using slow merge"); - partitionLengths = mergeSpillsWithFileStream(spills, mapWriter, compressionCodec); + partitionLengths = mergeSpillsWithFileStream(spills, outputFile, compressionCodec); } // When closing an UnsafeShuffleExternalSorter that has already spilled once but also has // in-memory records, we write out the in-memory records to a file but do not count that @@ -317,9 +331,13 @@ private long[] mergeSpills(SpillInfo[] spills, // to be counted as shuffle write, but this will lead to double-counting of the final // SpillInfo's bytes. writeMetrics.decBytesWritten(spills[spills.length - 1].file.length()); + writeMetrics.incBytesWritten(outputFile.length()); return partitionLengths; } } catch (IOException e) { + if (outputFile.exists() && !outputFile.delete()) { + logger.error("Unable to delete output file {}", outputFile.getPath()); + } throw e; } } @@ -327,71 +345,73 @@ private long[] mergeSpills(SpillInfo[] spills, /** * Merges spill files using Java FileStreams. This code path is typically slower than * the NIO-based merge, {@link UnsafeShuffleWriter#mergeSpillsWithTransferTo(SpillInfo[], - * ShuffleMapOutputWriter)}, and it's mostly used in cases where the IO compression codec - * does not support concatenation of compressed data, when encryption is enabled, or when - * users have explicitly disabled use of {@code transferTo} in order to work around kernel bugs. + * File)}, and it's mostly used in cases where the IO compression codec does not support + * concatenation of compressed data, when encryption is enabled, or when users have + * explicitly disabled use of {@code transferTo} in order to work around kernel bugs. * This code path might also be faster in cases where individual partition size in a spill * is small and UnsafeShuffleWriter#mergeSpillsWithTransferTo method performs many small * disk ios which is inefficient. In those case, Using large buffers for input and output * files helps reducing the number of disk ios, making the file merging faster. * * @param spills the spills to merge. - * @param mapWriter the map output writer to use for output. + * @param outputFile the file to write the merged data to. * @param compressionCodec the IO compression codec, or null if shuffle compression is disabled. * @return the partition lengths in the merged file. */ private long[] mergeSpillsWithFileStream( SpillInfo[] spills, - ShuffleMapOutputWriter mapWriter, + File outputFile, @Nullable CompressionCodec compressionCodec) throws IOException { + assert (spills.length >= 2); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final InputStream[] spillInputStreams = new InputStream[spills.length]; + final OutputStream bos = new BufferedOutputStream( + new FileOutputStream(outputFile), + outputBufferSizeInBytes); + // Use a counting output stream to avoid having to close the underlying file and ask + // the file system for its size after each partition is written. + final CountingOutputStream mergedFileOutputStream = new CountingOutputStream(bos); + boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputStreams[i] = new NioBufferedFileInputStream( - spills[i].file, - inputBufferSizeInBytes); + spills[i].file, + inputBufferSizeInBytes); } for (int partition = 0; partition < numPartitions; partition++) { - boolean copyThrewExecption = true; - ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); - OutputStream partitionOutput = null; - try { - partitionOutput = writer.openStream(); - partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); - if (compressionCodec != null) { - partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); - } - for (int i = 0; i < spills.length; i++) { - final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - - if (partitionLengthInSpill > 0) { - InputStream partitionInputStream = null; - try { - partitionInputStream = new LimitedInputStream(spillInputStreams[i], - partitionLengthInSpill, false); - partitionInputStream = blockManager.serializerManager().wrapForEncryption( - partitionInputStream); - if (compressionCodec != null) { - partitionInputStream = compressionCodec.compressedInputStream( - partitionInputStream); - } - ByteStreams.copy(partitionInputStream, partitionOutput); - } finally { - partitionInputStream.close(); + final long initialFileLength = mergedFileOutputStream.getByteCount(); + // Shield the underlying output stream from close() and flush() calls, so that we can close + // the higher level streams to make sure all data is really flushed and internal state is + // cleaned. + OutputStream partitionOutput = new CloseAndFlushShieldOutputStream( + new TimeTrackingOutputStream(writeMetrics, mergedFileOutputStream)); + partitionOutput = blockManager.serializerManager().wrapForEncryption(partitionOutput); + if (compressionCodec != null) { + partitionOutput = compressionCodec.compressedOutputStream(partitionOutput); + } + for (int i = 0; i < spills.length; i++) { + final long partitionLengthInSpill = spills[i].partitionLengths[partition]; + if (partitionLengthInSpill > 0) { + InputStream partitionInputStream = new LimitedInputStream(spillInputStreams[i], + partitionLengthInSpill, false); + try { + partitionInputStream = blockManager.serializerManager().wrapForEncryption( + partitionInputStream); + if (compressionCodec != null) { + partitionInputStream = compressionCodec.compressedInputStream(partitionInputStream); } + ByteStreams.copy(partitionInputStream, partitionOutput); + } finally { + partitionInputStream.close(); } - copyThrewExecption = false; } - } finally { - Closeables.close(partitionOutput, copyThrewExecption); } - long numBytesWritten = writer.getNumBytesWritten(); - partitionLengths[partition] = numBytesWritten; - writeMetrics.incBytesWritten(numBytesWritten); + partitionOutput.flush(); + partitionOutput.close(); + partitionLengths[partition] = (mergedFileOutputStream.getByteCount() - initialFileLength); } threwException = false; } finally { @@ -400,6 +420,7 @@ private long[] mergeSpillsWithFileStream( for (InputStream stream : spillInputStreams) { Closeables.close(stream, threwException); } + Closeables.close(mergedFileOutputStream, threwException); } return partitionLengths; } @@ -409,49 +430,54 @@ private long[] mergeSpillsWithFileStream( * This is only safe when the IO compression codec and serializer support concatenation of * serialized streams. * - * @param spills the spills to merge. - * @param mapWriter the map output writer to use for output. * @return the partition lengths in the merged file. */ - private long[] mergeSpillsWithTransferTo( - SpillInfo[] spills, - ShuffleMapOutputWriter mapWriter) throws IOException { + private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) throws IOException { + assert (spills.length >= 2); final int numPartitions = partitioner.numPartitions(); final long[] partitionLengths = new long[numPartitions]; final FileChannel[] spillInputChannels = new FileChannel[spills.length]; final long[] spillInputChannelPositions = new long[spills.length]; + FileChannel mergedFileOutputChannel = null; boolean threwException = true; try { for (int i = 0; i < spills.length; i++) { spillInputChannels[i] = new FileInputStream(spills[i].file).getChannel(); } + // This file needs to opened in append mode in order to work around a Linux kernel bug that + // affects transferTo; see SPARK-3948 for more details. + mergedFileOutputChannel = new FileOutputStream(outputFile, true).getChannel(); + + long bytesWrittenToMergedFile = 0; for (int partition = 0; partition < numPartitions; partition++) { - boolean copyThrewExecption = true; - ShufflePartitionWriter writer = mapWriter.getPartitionWriter(partition); - TransferrableWritableByteChannel partitionChannel = null; - try { - partitionChannel = writer instanceof SupportsTransferTo ? - ((SupportsTransferTo) writer).openTransferrableChannel() - : new DefaultTransferrableWritableByteChannel( - Channels.newChannel(writer.openStream())); - for (int i = 0; i < spills.length; i++) { - long partitionLengthInSpill = 0L; - partitionLengthInSpill += spills[i].partitionLengths[partition]; - final FileChannel spillInputChannel = spillInputChannels[i]; - final long writeStartTime = System.nanoTime(); - partitionChannel.transferFrom( - spillInputChannel, spillInputChannelPositions[i], partitionLengthInSpill); - spillInputChannelPositions[i] += partitionLengthInSpill; - writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); - } - copyThrewExecption = false; - } finally { - Closeables.close(partitionChannel, copyThrewExecption); + for (int i = 0; i < spills.length; i++) { + final long partitionLengthInSpill = spills[i].partitionLengths[partition]; + final FileChannel spillInputChannel = spillInputChannels[i]; + final long writeStartTime = System.nanoTime(); + Utils.copyFileStreamNIO( + spillInputChannel, + mergedFileOutputChannel, + spillInputChannelPositions[i], + partitionLengthInSpill); + spillInputChannelPositions[i] += partitionLengthInSpill; + writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); + bytesWrittenToMergedFile += partitionLengthInSpill; + partitionLengths[partition] += partitionLengthInSpill; } - long numBytes = writer.getNumBytesWritten(); - partitionLengths[partition] = numBytes; - writeMetrics.incBytesWritten(numBytes); + } + // Check the position after transferTo loop to see if it is in the right position and raise an + // exception if it is incorrect. The position will not be increased to the expected length + // after calling transferTo in kernel version 2.6.32. This issue is described at + // https://bugs.openjdk.java.net/browse/JDK-7052359 and SPARK-3948. + if (mergedFileOutputChannel.position() != bytesWrittenToMergedFile) { + throw new IOException( + "Current position " + mergedFileOutputChannel.position() + " does not equal expected " + + "position " + bytesWrittenToMergedFile + " after transferTo. Please check your kernel" + + " version to see if it is 2.6.32, as there is a kernel bug which will lead to " + + "unexpected behavior when using transferTo. You can set spark.file.transferTo=false " + + "to disable this NIO feature." + ); } threwException = false; } finally { @@ -461,6 +487,7 @@ private long[] mergeSpillsWithTransferTo( assert(spillInputChannelPositions[i] == spills[i].file.length()); Closeables.close(spillInputChannels[i], threwException); } + Closeables.close(mergedFileOutputChannel, threwException); } return partitionLengths; } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 5fa9296b022c..3d03cecf9e9d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -140,13 +140,13 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager case unsafeShuffleHandle: SerializedShuffleHandle[K @unchecked, V @unchecked] => new UnsafeShuffleWriter( env.blockManager, + shuffleBlockResolver, context.taskMemoryManager(), unsafeShuffleHandle, mapId, context, env.conf, - metrics, - shuffleExecutorComponents.writes()) + metrics) case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new BypassMergeSortShuffleWriter( env.blockManager, diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index 64227e4aefbb..88125a6b93ad 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -19,10 +19,8 @@ import java.io.*; import java.nio.ByteBuffer; -import java.nio.file.Files; import java.util.*; -import org.mockito.stubbing.Answer; import scala.Option; import scala.Product2; import scala.Tuple2; @@ -41,7 +39,6 @@ import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; -import org.apache.spark.TaskContext$; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.io.CompressionCodec$; @@ -56,7 +53,6 @@ import org.apache.spark.security.CryptoStreamUtils; import org.apache.spark.serializer.*; import org.apache.spark.shuffle.IndexShuffleBlockResolver; -import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport; import org.apache.spark.storage.*; import org.apache.spark.util.Utils; @@ -69,7 +65,6 @@ public class UnsafeShuffleWriterSuite { - static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; static final int NUM_PARTITITONS = 4; TestMemoryManager memoryManager; TaskMemoryManager taskMemoryManager; @@ -90,7 +85,6 @@ public class UnsafeShuffleWriterSuite { @After public void tearDown() { - TaskContext$.MODULE$.unset(); Utils.deleteRecursively(tempDir); final long leakedMemory = taskMemoryManager.cleanUpAllAllocatedMemory(); if (leakedMemory != 0) { @@ -138,28 +132,14 @@ public void setUp() throws IOException { }); when(shuffleBlockResolver.getDataFile(anyInt(), anyInt())).thenReturn(mergedOutputFile); - - Answer renameTempAnswer = invocationOnMock -> { + doAnswer(invocationOnMock -> { partitionSizesInMergedFile = (long[]) invocationOnMock.getArguments()[2]; File tmp = (File) invocationOnMock.getArguments()[3]; - if (!mergedOutputFile.delete()) { - throw new RuntimeException("Failed to delete old merged output file."); - } - if (tmp != null) { - Files.move(tmp.toPath(), mergedOutputFile.toPath()); - } else if (!mergedOutputFile.createNewFile()) { - throw new RuntimeException("Failed to create empty merged output file."); - } + mergedOutputFile.delete(); + tmp.renameTo(mergedOutputFile); return null; - }; - - doAnswer(renameTempAnswer) - .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); - - doAnswer(renameTempAnswer) - .when(shuffleBlockResolver) - .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), eq(null)); + }).when(shuffleBlockResolver) + .writeIndexFileAndCommit(anyInt(), anyInt(), any(long[].class), any(File.class)); when(diskBlockManager.createTempShuffleBlock()).thenAnswer(invocationOnMock -> { TempShuffleBlockId blockId = new TempShuffleBlockId(UUID.randomUUID()); @@ -171,11 +151,6 @@ public void setUp() throws IOException { when(taskContext.taskMetrics()).thenReturn(taskMetrics); when(shuffleDep.serializer()).thenReturn(serializer); when(shuffleDep.partitioner()).thenReturn(hashPartitioner); - when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager); - when(blockManager.shuffleServerId()).thenReturn(BlockManagerId.apply( - "0", "localhost", 9099, Option.empty())); - - TaskContext$.MODULE$.setTaskContext(taskContext); } private UnsafeShuffleWriter createWriter( @@ -183,13 +158,14 @@ private UnsafeShuffleWriter createWriter( conf.set("spark.file.transferTo", String.valueOf(transferToEnabled)); return new UnsafeShuffleWriter<>( blockManager, - taskMemoryManager, + shuffleBlockResolver, + taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); + taskContext.taskMetrics().shuffleWriteMetrics() + ); } private void assertSpillFilesWereCleanedUp() { @@ -468,10 +444,10 @@ public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() thro } private void writeEnoughRecordsToTriggerSortBufferExpansionAndSpill() throws Exception { - memoryManager.limit(DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); + memoryManager.limit(UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE * 16); final UnsafeShuffleWriter writer = createWriter(false); final ArrayList> dataToWrite = new ArrayList<>(); - for (int i = 0; i < DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { + for (int i = 0; i < UnsafeShuffleWriter.DEFAULT_INITIAL_SORT_BUFFER_SIZE + 1; i++) { dataToWrite.add(new Tuple2<>(i, i)); } writer.write(dataToWrite.iterator()); @@ -543,13 +519,13 @@ public void testPeakMemoryUsed() throws Exception { final UnsafeShuffleWriter writer = new UnsafeShuffleWriter<>( blockManager, - taskMemoryManager, + shuffleBlockResolver, + taskMemoryManager, new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, conf, - taskContext.taskMetrics().shuffleWriteMetrics(), - new DefaultShuffleWriteSupport(conf, shuffleBlockResolver)); + taskContext.taskMetrics().shuffleWriteMetrics()); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. From a8558af85ca3fabbe49c0d0393afaad5ddbd9a29 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 26 Jun 2019 18:25:01 -0700 Subject: [PATCH 11/32] Revert changes for SortShuffleWriter --- .../shuffle/sort/SortShuffleManager.scala | 3 +- .../shuffle/sort/SortShuffleWriter.scala | 23 +++-- .../spark/storage/DiskBlockObjectWriter.scala | 4 +- .../util/collection/ExternalSorter.scala | 95 ++----------------- .../spark/util/collection/PairsWriter.scala | 23 ----- .../ShufflePartitionPairsWriter.scala | 91 ------------------ .../WritablePartitionedPairCollection.scala | 4 +- 7 files changed, 26 insertions(+), 217 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala delete mode 100644 core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 3d03cecf9e9d..49f43be9e2ac 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -156,8 +156,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager metrics, shuffleExecutorComponents.writes()) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => - new SortShuffleWriter( - shuffleBlockResolver, other, mapId, context, shuffleExecutorComponents.writes()) + new SortShuffleWriter(shuffleBlockResolver, other, mapId, context) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 62316f384b64..16058de8bf3f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,18 +18,18 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ -import org.apache.spark.api.shuffle.ShuffleWriteSupport import org.apache.spark.internal.{config, 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.util.Utils import org.apache.spark.util.collection.ExternalSorter private[spark] class SortShuffleWriter[K, V, C]( shuffleBlockResolver: IndexShuffleBlockResolver, handle: BaseShuffleHandle[K, V, C], mapId: Int, - context: TaskContext, - writeSupport: ShuffleWriteSupport) + context: TaskContext) extends ShuffleWriter[K, V] with Logging { private val dep = handle.dependency @@ -64,11 +64,18 @@ private[spark] class SortShuffleWriter[K, V, C]( // Don't bother including the time to open the merged output file in the shuffle write time, // because it just opens a single file, so is typically too fast to measure accurately // (see SPARK-3570). - val mapOutputWriter = writeSupport.createMapOutputWriter( - dep.shuffleId, mapId, dep.partitioner.numPartitions) - val partitionLengths = sorter.writePartitionedMapOutput(dep.shuffleId, mapId, mapOutputWriter) - mapOutputWriter.commitAllPartitions() - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) + val tmp = Utils.tempFileWith(output) + try { + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val partitionLengths = sorter.writePartitionedFile(blockId, tmp) + shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + } finally { + if (tmp.exists() && !tmp.delete()) { + logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") + } + } } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index f9f4e3594e4f..17390f9c60e7 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -24,7 +24,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.util.Utils -import org.apache.spark.util.collection.PairsWriter /** * A class for writing JVM objects directly to a file on disk. This class allows data to be appended @@ -47,8 +46,7 @@ private[spark] class DiskBlockObjectWriter( writeMetrics: ShuffleWriteMetricsReporter, val blockId: BlockId = null) extends OutputStream - with Logging - with PairsWriter { + with Logging { /** * Guards against close calls, e.g. from a wrapping stream. diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 13132c2801ed..3f3b7d20eb16 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -26,11 +26,10 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams import org.apache.spark._ -import org.apache.spark.api.shuffle.{ShuffleMapOutputWriter, ShufflePartitionWriter} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer._ -import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -671,9 +670,11 @@ private[spark] class ExternalSorter[K, V, C]( } /** - * TODO remove this, as this is only used by UnsafeRowSerializerSuite in the SQL project. - * We should figure out an alternative way to test that so that we can remove this otherwise - * unused code path. + * Write all the data added into this ExternalSorter into a file in the disk store. This is + * called by the SortShuffleWriter. + * + * @param blockId block ID to write to. The index file will be blockId.name + ".index". + * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) */ def writePartitionedFile( blockId: BlockId, @@ -717,88 +718,6 @@ private[spark] class ExternalSorter[K, V, C]( lengths } - /** - * Write all the data added into this ExternalSorter into a map output writer that pushes bytes - * to some arbitrary backing store. This is called by the SortShuffleWriter. - * - * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) - */ - def writePartitionedMapOutput( - shuffleId: Int, mapId: Int, mapOutputWriter: ShuffleMapOutputWriter): Array[Long] = { - // Track location of each range in the map output - val lengths = new Array[Long](numPartitions) - if (spills.isEmpty) { - // Case where we only have in-memory data - val collection = if (aggregator.isDefined) map else buffer - val it = collection.destructiveSortedWritablePartitionedIterator(comparator) - while (it.hasNext()) { - val partitionId = it.nextPartition() - var partitionWriter: ShufflePartitionWriter = null - var partitionPairsWriter: ShufflePartitionPairsWriter = null - try { - partitionWriter = mapOutputWriter.getPartitionWriter(partitionId) - val blockId = ShuffleBlockId(shuffleId, mapId, partitionId) - partitionPairsWriter = new ShufflePartitionPairsWriter( - partitionWriter, - serializerManager, - serInstance, - blockId, - context.taskMetrics().shuffleWriteMetrics) - while (it.hasNext && it.nextPartition() == partitionId) { - it.writeNext(partitionPairsWriter) - } - } finally { - if (partitionPairsWriter != null) { - partitionPairsWriter.close() - } - } - if (partitionWriter != null) { - lengths(partitionId) = partitionWriter.getNumBytesWritten - } - } - } else { - // We must perform merge-sort; get an iterator by partition and write everything directly. - for ((id, elements) <- this.partitionedIterator) { - // The contract for the plugin is that we will ask for a writer for every partition - // even if it's empty. However, the external sorter will return non-contiguous - // partition ids. So this loop "backfills" the empty partitions that form the gaps. - - // The algorithm as a whole is correct because the partition ids are returned by the - // iterator in ascending order. - val blockId = ShuffleBlockId(shuffleId, mapId, id) - var partitionWriter: ShufflePartitionWriter = null - var partitionPairsWriter: ShufflePartitionPairsWriter = null - try { - partitionWriter = mapOutputWriter.getPartitionWriter(id) - partitionPairsWriter = new ShufflePartitionPairsWriter( - partitionWriter, - serializerManager, - serInstance, - blockId, - context.taskMetrics().shuffleWriteMetrics) - if (elements.hasNext) { - for (elem <- elements) { - partitionPairsWriter.write(elem._1, elem._2) - } - } - } finally { - if (partitionPairsWriter!= null) { - partitionPairsWriter.close() - } - } - if (partitionWriter != null) { - lengths(id) = partitionWriter.getNumBytesWritten - } - } - } - - context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) - context.taskMetrics().incDiskBytesSpilled(diskBytesSpilled) - context.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes) - - lengths - } - def stop(): Unit = { spills.foreach(s => s.file.delete()) spills.clear() @@ -862,7 +781,7 @@ private[spark] class ExternalSorter[K, V, C]( val inMemoryIterator = new WritablePartitionedIterator { private[this] var cur = if (upstream.hasNext) upstream.next() else null - def writeNext(writer: PairsWriter): Unit = { + def writeNext(writer: DiskBlockObjectWriter): Unit = { writer.write(cur._1._2, cur._2) cur = if (upstream.hasNext) upstream.next() else null } diff --git a/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala deleted file mode 100644 index 9d7c209f242e..000000000000 --- a/core/src/main/scala/org/apache/spark/util/collection/PairsWriter.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.util.collection - -private[spark] trait PairsWriter { - - def write(key: Any, value: Any): Unit -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala b/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala deleted file mode 100644 index 8538a78b377c..000000000000 --- a/core/src/main/scala/org/apache/spark/util/collection/ShufflePartitionPairsWriter.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * 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.util.collection - -import java.io.{Closeable, FilterOutputStream, OutputStream} - -import org.apache.spark.api.shuffle.ShufflePartitionWriter -import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter -import org.apache.spark.storage.BlockId - -/** - * A key-value writer inspired by {@link DiskBlockObjectWriter} that pushes the bytes to an - * arbitrary partition writer instead of writing to local disk through the block manager. - */ -private[spark] class ShufflePartitionPairsWriter( - partitionWriter: ShufflePartitionWriter, - serializerManager: SerializerManager, - serializerInstance: SerializerInstance, - blockId: BlockId, - writeMetrics: ShuffleWriteMetricsReporter) - extends PairsWriter with Closeable { - - private var isOpen = false - private var partitionStream: OutputStream = _ - private var wrappedStream: OutputStream = _ - private var objOut: SerializationStream = _ - private var numRecordsWritten = 0 - private var curNumBytesWritten = 0L - - override def write(key: Any, value: Any): Unit = { - if (!isOpen) { - open() - isOpen = true - } - objOut.writeKey(key) - objOut.writeValue(value) - writeMetrics.incRecordsWritten(1) - } - - private def open(): Unit = { - partitionStream = partitionWriter.openStream - wrappedStream = serializerManager.wrapStream(blockId, partitionStream) - objOut = serializerInstance.serializeStream(wrappedStream) - } - - override def close(): Unit = { - if (isOpen) { - objOut.close() - objOut = null - wrappedStream = null - partitionStream = null - isOpen = false - updateBytesWritten() - } - } - - /** - * Notify the writer that a record worth of bytes has been written with OutputStream#write. - */ - private def recordWritten(): Unit = { - numRecordsWritten += 1 - writeMetrics.incRecordsWritten(1) - - if (numRecordsWritten % 16384 == 0) { - updateBytesWritten() - } - } - - private def updateBytesWritten(): Unit = { - val numBytesWritten = partitionWriter.getNumBytesWritten - val bytesWrittenDiff = numBytesWritten - curNumBytesWritten - writeMetrics.incBytesWritten(bytesWrittenDiff) - curNumBytesWritten = numBytesWritten - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala index da8d58d05b6b..dd7f68fd038d 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -52,7 +52,7 @@ private[spark] trait WritablePartitionedPairCollection[K, V] { new WritablePartitionedIterator { private[this] var cur = if (it.hasNext) it.next() else null - def writeNext(writer: PairsWriter): Unit = { + def writeNext(writer: DiskBlockObjectWriter): Unit = { writer.write(cur._1._2, cur._2) cur = if (it.hasNext) it.next() else null } @@ -89,7 +89,7 @@ private[spark] object WritablePartitionedPairCollection { * has an associated partition. */ private[spark] trait WritablePartitionedIterator { - def writeNext(writer: PairsWriter): Unit + def writeNext(writer: DiskBlockObjectWriter): Unit def hasNext(): Boolean From 806d7bb3bf615f045fc589ba950ea4ec0d46149f Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 26 Jun 2019 18:32:55 -0700 Subject: [PATCH 12/32] Revert a bunch of other stuff --- .../apache/spark/storage/BlockManagerId.scala | 4 +--- .../test/org/apache/spark/JavaAPISuite.java | 4 +--- .../scala/org/apache/spark/ShuffleSuite.scala | 14 +++++-------- .../spark/scheduler/MapStatusSuite.scala | 20 ++++++++----------- .../scheduler/SchedulerIntegrationSuite.scala | 3 +-- .../serializer/KryoSerializerSuite.scala | 6 ++---- .../shuffle/sort/SortShuffleWriterSuite.scala | 19 +++--------------- 7 files changed, 21 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 97b99e08d9ca..d188bdd912e5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -132,14 +132,12 @@ private[spark] object BlockManagerId { getCachedBlockManagerId(obj) } - val blockManagerIdCacheSize = 10000 - /** * The max cache size is hardcoded to 10000, since the size of a BlockManagerId * object is about 48B, the total memory cost should be below 1MB which is feasible. */ val blockManagerIdCache = CacheBuilder.newBuilder() - .maximumSize(blockManagerIdCacheSize) + .maximumSize(10000) .build(new CacheLoader[BlockManagerId, BlockManagerId]() { override def load(id: BlockManagerId) = id }) diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 08f0d65abff1..a8252e03b5c1 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -94,9 +94,7 @@ public void setUp() { @After public void tearDown() { - if (sc != null) { - sc.stop(); - } + sc.stop(); sc = null; } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 90c790cefcca..8b1084a8edc7 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListene import org.apache.spark.serializer.KryoSerializer import org.apache.spark.shuffle.ShuffleWriter import org.apache.spark.storage.{ShuffleBlockId, ShuffleDataBlockId, ShuffleIndexBlockId} -import org.apache.spark.util.MutablePair +import org.apache.spark.util.{MutablePair, Utils} abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkContext { @@ -368,7 +368,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem) val writer1 = manager.getWriter[Int, Int]( shuffleHandle, 0, context1, context1.taskMetrics.shuffleWriteMetrics) - val data1 = (1 to 10).map { x => x -> x } + val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, // just to simulate the fact that the records may get written differently @@ -383,17 +383,13 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // simultaneously, and everything is still OK def writeAndClose( - writer: ShuffleWriter[Int, Int], - taskContext: TaskContext)( + writer: ShuffleWriter[Int, Int])( iter: Iterator[(Int, Int)]): Option[MapStatus] = { - TaskContext.setTaskContext(taskContext) val files = writer.write(iter) - val status = writer.stop(true) - TaskContext.unset - status + writer.stop(true) } val interleaver = new InterleaveIterators( - data1, writeAndClose(writer1, context1), data2, writeAndClose(writer2, context2)) + data1, writeAndClose(writer1), data2, writeAndClose(writer2)) val (mapOutput1, mapOutput2) = interleaver.run() // check that we can read the map output and it has the right data diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index d7304ce714c4..c1e7fb9a1db1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -61,10 +61,7 @@ class MapStatusSuite extends SparkFunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val bmId = BlockManagerId("a", "b", 10) - val status = MapStatus( - bmId, - sizes) + val status = MapStatus(BlockManagerId("a", "b", 10), sizes) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -89,11 +86,11 @@ class MapStatusSuite extends SparkFunSuite { test("HighlyCompressedMapStatus: estimated size should be the average non-empty block size") { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) - val bmId = BlockManagerId("a", "b", 10) - val status = MapStatus(bmId, sizes) + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) - assert(status1.location == bmId) + assert(status1.location == loc) for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -111,11 +108,11 @@ class MapStatusSuite extends SparkFunSuite { val sizes = (0L to 3000L).toArray val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length - val bmId = BlockManagerId("a", "b", 10) - val status = MapStatus(bmId, sizes) + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) - assert(status1.location === bmId) + assert(status1.location == loc) for (i <- 0 until threshold) { val estimate = status1.getSizeForBlock(i) if (sizes(i) > 0) { @@ -168,8 +165,7 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray - val bmId = BlockManagerId("exec-0", "host-0", 100) - val status1 = MapStatus(bmId, sizes) + val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 7a1f5b690ab8..577d77e890d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -191,8 +191,7 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa shuffleId <- shuffleIds reduceIdx <- (0 until nParts) } { - val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId( - shuffleId, reduceIdx) + val statuses = taskScheduler.mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceIdx) // really we should have already thrown an exception rather than fail either of these // asserts, but just to be extra defensive let's double check the statuses are OK assert(statuses != null) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 8740ca09515d..2442670b6d3f 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") @@ -350,10 +350,8 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val ser = new KryoSerializer(conf).newInstance() val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) - val bmId = BlockManagerId("exec-1", "host", 1234) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize(HighlyCompressedMapStatus( - bmId, blockSizes)) + ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala index e99e179821c5..690bcd990525 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -21,12 +21,10 @@ import org.mockito.Mockito._ import org.mockito.MockitoAnnotations import org.scalatest.Matchers -import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite, TaskContext} -import org.apache.spark.api.shuffle.ShuffleWriteSupport +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} -import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport import org.apache.spark.util.Utils @@ -37,11 +35,9 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with private var shuffleHandle: BaseShuffleHandle[Int, Int, Int] = _ private val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) private val serializer = new JavaSerializer(conf) - private var writeSupport: ShuffleWriteSupport = _ override def beforeEach(): Unit = { super.beforeEach() - writeSupport = new DefaultShuffleWriteSupport(conf, shuffleBlockResolver) MockitoAnnotations.initMocks(this) val partitioner = new Partitioner() { def numPartitions = numMaps @@ -57,11 +53,6 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with } } - override def afterEach(): Unit = { - super.afterEach() - TaskContext.unset() - } - override def afterAll(): Unit = { try { shuffleBlockResolver.stop() @@ -72,13 +63,11 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with test("write empty iterator") { val context = MemoryTestingUtils.fakeTaskContext(sc.env) - TaskContext.setTaskContext(context) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, mapId = 1, - context, - writeSupport) + context) writer.write(Iterator.empty) writer.stop(success = true) val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 1) @@ -90,14 +79,12 @@ class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with test("write with some records") { val context = MemoryTestingUtils.fakeTaskContext(sc.env) - TaskContext.setTaskContext(context) val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) val writer = new SortShuffleWriter[Int, Int, Int]( shuffleBlockResolver, shuffleHandle, mapId = 2, - context, - writeSupport) + context) writer.write(records.toIterator) writer.stop(success = true) val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 2) From 3167030e4ac58b5ca86d5000e191b7b1c14f5e3a Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 26 Jun 2019 18:34:38 -0700 Subject: [PATCH 13/32] More reverts --- .../apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 1aa699e7ad6a..6d2ef17a7a79 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -102,8 +102,7 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext // Make a mocked MapOutputTracker for the shuffle reader to use to determine what // shuffle data to read. val mapOutputTracker = mock(classOf[MapOutputTracker]) - when(mapOutputTracker.getMapSizesByExecutorId( - shuffleId, reduceId, reduceId + 1)).thenReturn { + when(mapOutputTracker.getMapSizesByExecutorId(shuffleId, reduceId, reduceId + 1)).thenReturn { // Test a scenario where all data is local, to avoid creating a bunch of additional mocks // for the code to read data over the network. val shuffleBlockIdsAndSizes = (0 until numMaps).map { mapId => From 70f59db45a4518159c613d3106533c4cc0ece05e Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 28 Jun 2019 14:58:07 -0700 Subject: [PATCH 14/32] Set task contexts in failing test --- .../scala/org/apache/spark/ShuffleSuite.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 8b1084a8edc7..33ee6e8a21ae 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -383,13 +383,19 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // simultaneously, and everything is still OK def writeAndClose( - writer: ShuffleWriter[Int, Int])( + writer: ShuffleWriter[Int, Int], + taskContext: TaskContext)( iter: Iterator[(Int, Int)]): Option[MapStatus] = { - val files = writer.write(iter) - writer.stop(true) + TaskContext.setTaskContext(taskContext) + try { + val files = writer.write(iter) + writer.stop(true) + } finally { + TaskContext.unset() + } } val interleaver = new InterleaveIterators( - data1, writeAndClose(writer1), data2, writeAndClose(writer2)) + data1, writeAndClose(writer1, context1), data2, writeAndClose(writer2, context2)) val (mapOutput1, mapOutput2) = interleaver.run() // check that we can read the map output and it has the right data @@ -405,8 +411,10 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val taskContext = new TaskContextImpl( 1, 0, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem) + TaskContext.setTaskContext(taskContext) val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, taskContext, metrics) + TaskContext.unset() val readData = reader.read().toIndexedSeq assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) From 3083d8686cf1e1650969dde3d0862bd8c53855f0 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 28 Jun 2019 15:26:42 -0700 Subject: [PATCH 15/32] Fix style --- .../apache/spark/api/shuffle/ShuffleMapOutputWriter.java | 1 - .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 1 - .../shuffle/sort/io/DefaultShuffleMapOutputWriter.java | 2 -- .../spark/shuffle/sort/io/DefaultShuffleWriteSupport.java | 8 ++++++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java index 6822ed05b911..0a95193b179f 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.spark.annotation.Experimental; -import org.apache.spark.api.java.Optional; /** * :: Experimental :: diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 85270f273a89..95b26030b816 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -39,7 +39,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.SupportsTransferTo; import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; import org.apache.spark.api.shuffle.ShufflePartitionWriter; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java index cf57ba6f9cfd..4d2967b251e7 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java @@ -28,7 +28,6 @@ import org.slf4j.LoggerFactory; import org.apache.spark.SparkConf; -import org.apache.spark.api.java.Optional; import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; import org.apache.spark.api.shuffle.ShufflePartitionWriter; import org.apache.spark.api.shuffle.SupportsTransferTo; @@ -36,7 +35,6 @@ import org.apache.spark.internal.config.package$; 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; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java index ff0b6dc370f7..8a583d1bd50a 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java @@ -22,7 +22,6 @@ import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; import org.apache.spark.api.shuffle.ShuffleWriteSupport; import org.apache.spark.shuffle.IndexShuffleBlockResolver; -import org.apache.spark.storage.BlockManagerId; public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { @@ -41,8 +40,13 @@ public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, int numPartitions) { + TaskContext taskContext = TaskContext.get(); + if (taskContext == null) { + throw new IllegalStateException( + "Task context must be set before creating a map output writer."); + } return new DefaultShuffleMapOutputWriter( shuffleId, mapId, numPartitions, - TaskContext.get().taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); + taskContext.taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); } } From 4c3d6926069bea6e89f1a13d8faa46ce1180fcf0 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 28 Jun 2019 15:59:57 -0700 Subject: [PATCH 16/32] Check for null on the block manager as well. --- .../shuffle/sort/io/DefaultShuffleExecutorComponents.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java index 76e87a674025..fb999ad028f0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java @@ -37,6 +37,9 @@ public DefaultShuffleExecutorComponents(SparkConf sparkConf) { @Override public void initializeExecutor(String appId, String execId) { blockManager = SparkEnv.get().blockManager(); + if (blockManager == null) { + throw new IllegalStateException("No blockManager available from the SparkEnv."); + } blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager); } From 2421c928dd35ccaa28825a96bcd3449e94383b59 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 1 Jul 2019 16:46:33 -0700 Subject: [PATCH 17/32] Add task attempt id in the APIs --- .../org/apache/spark/api/shuffle/ShuffleWriteSupport.java | 1 + .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 5 ++++- .../spark/shuffle/sort/io/DefaultShuffleWriteSupport.java | 1 + .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 1 + .../shuffle/sort/BypassMergeSortShuffleWriterSuite.scala | 5 +++++ 5 files changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java index 7e2b6cf4133f..7ee1d8a55407 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java @@ -32,5 +32,6 @@ public interface ShuffleWriteSupport { ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, + long mapTaskAttemptId, int numPartitions) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 95b26030b816..ec828a150235 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -87,6 +87,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; private final int mapId; + private final long mapTaskAttemptId; private final Serializer serializer; private final ShuffleWriteSupport shuffleWriteSupport; @@ -107,6 +108,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { BlockManager blockManager, BypassMergeSortShuffleHandle handle, int mapId, + long mapTaskAttemptId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, ShuffleWriteSupport shuffleWriteSupport) { @@ -116,6 +118,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); this.mapId = mapId; + this.mapTaskAttemptId = mapTaskAttemptId; this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); @@ -128,7 +131,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleWriteSupport - .createMapOutputWriter(shuffleId, mapId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); try { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java index 8a583d1bd50a..be392ecbe825 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java @@ -39,6 +39,7 @@ public DefaultShuffleWriteSupport( public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, + long mapTaskAttemptId, int numPartitions) { TaskContext taskContext = TaskContext.get(); if (taskContext == null) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 49f43be9e2ac..a2cd942849c8 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -152,6 +152,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager env.blockManager, bypassMergeSortHandle, mapId, + context.taskAttemptId(), env.conf, metrics, shuffleExecutorComponents.writes()) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index b16bf5194eeb..02811cf67e4d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -148,6 +148,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId + 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, writeSupport @@ -173,6 +174,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId + 0L, transferConf, taskContext.taskMetrics().shuffleWriteMetrics, writeSupport @@ -197,6 +199,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId + 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, writeSupport @@ -232,6 +235,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId + 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, writeSupport @@ -254,6 +258,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId + 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, writeSupport From 982f207e2f3f78b1f3eb81e13a0dacca4ba9137f Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 8 Jul 2019 15:58:18 -0700 Subject: [PATCH 18/32] Address comments --- .../api/shuffle/ShuffleMapOutputWriter.java | 37 ----------- .../api}/ShuffleDataIO.java | 21 +++++- .../api}/ShuffleExecutorComponents.java | 20 +++++- .../shuffle/api/ShuffleMapOutputWriter.java | 65 +++++++++++++++++++ .../api}/ShufflePartitionWriter.java | 6 +- .../api}/ShuffleWriteSupport.java | 13 +++- .../api}/SupportsTransferTo.java | 2 +- .../TransferrableWritableByteChannel.java | 5 +- .../sort/BypassMergeSortShuffleWriter.java | 17 ++--- ...faultTransferrableWritableByteChannel.java | 5 +- ...ataIO.java => LocalDiskShuffleDataIO.java} | 14 ++-- ...> LocalDiskShuffleExecutorComponents.java} | 10 +-- ...a => LocalDiskShuffleMapOutputWriter.java} | 25 ++++--- ...java => LocalDiskShuffleWriteSupport.java} | 10 +-- .../spark/internal/config/package.scala | 4 +- .../shuffle/sort/SortShuffleManager.scala | 4 +- .../BypassMergeSortShuffleWriterSuite.scala | 7 +- ...ocalDiskShuffleMapOutputWriterSuite.scala} | 10 +-- 18 files changed, 181 insertions(+), 94 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/ShuffleDataIO.java (55%) rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/ShuffleExecutorComponents.java (57%) create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/ShufflePartitionWriter.java (85%) rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/ShuffleWriteSupport.java (68%) rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/SupportsTransferTo.java (98%) rename core/src/main/java/org/apache/spark/{api/shuffle => shuffle/api}/TransferrableWritableByteChannel.java (91%) rename core/src/main/java/org/apache/spark/shuffle/sort/io/{DefaultShuffleDataIO.java => LocalDiskShuffleDataIO.java} (67%) rename core/src/main/java/org/apache/spark/shuffle/sort/io/{DefaultShuffleExecutorComponents.java => LocalDiskShuffleExecutorComponents.java} (83%) rename core/src/main/java/org/apache/spark/shuffle/sort/io/{DefaultShuffleMapOutputWriter.java => LocalDiskShuffleMapOutputWriter.java} (89%) rename core/src/main/java/org/apache/spark/shuffle/sort/io/{DefaultShuffleWriteSupport.java => LocalDiskShuffleWriteSupport.java} (86%) rename core/src/test/scala/org/apache/spark/shuffle/sort/io/{DefaultShuffleMapOutputWriterSuite.scala => LocalDiskShuffleMapOutputWriterSuite.scala} (96%) diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java deleted file mode 100644 index 0a95193b179f..000000000000 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleMapOutputWriter.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.api.shuffle; - -import java.io.IOException; - -import org.apache.spark.annotation.Experimental; - -/** - * :: Experimental :: - * An interface for creating and managing shuffle partition writers - * - * @since 3.0.0 - */ -@Experimental -public interface ShuffleMapOutputWriter { - ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException; - - void commitAllPartitions() throws IOException; - - void abort(Throwable error) throws IOException; -} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java similarity index 55% rename from core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java rename to core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index 4cb40f6dd00b..72723eef116c 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -15,17 +15,32 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import org.apache.spark.annotation.Experimental; /** * :: Experimental :: - * An interface for launching Shuffle related components - * + * An interface for launching Shuffle related components. + *

+ * 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. + *

+ * Alternative implementations of shuffle data storage can be loaded via setting + * spark.shuffle.io.plugin.class. * @since 3.0.0 */ @Experimental public interface ShuffleDataIO { + + /** + * Called once on executor processes to bootstrap the shuffle data storage modules that + * are only invoked on the executors. + *

+ * At this point, this module is responsible for reading and writing shuffle data bytes + * from the backing store. + */ ShuffleExecutorComponents executor(); } diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java similarity index 57% rename from core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java rename to core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 4fc20bad9938..dfef914b2949 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -15,19 +15,35 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import org.apache.spark.annotation.Experimental; /** * :: Experimental :: - * An interface for building shuffle support for Executors + * An interface for building shuffle support for Executors. + *

+ * At present, submodules returned by this plugin are responsible for managing the + * writing of shuffle bytes (via {@link ShuffleWriteSupport} and the reading of + * shuffle bytes (to be added in subsequent patches). * * @since 3.0.0 */ @Experimental 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); + /** + * Returns the modules that are responsible for persisting shuffle data to the backing + * store. + *

+ * This may be called multiple times on each executor. Implementations should not make + * any assumptions about the lifetime of the returned module. + */ ShuffleWriteSupport writes(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java new file mode 100644 index 000000000000..744ac22ff356 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -0,0 +1,65 @@ +/* + * 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.Experimental; + +/** + * :: Experimental :: + * 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 + */ +@Experimental +public interface ShuffleMapOutputWriter { + + /** + * Creates a writer that can open an output stream to persist bytes for a given chunk of + * a map task. + *

+ * The chunk corresponds to bytes in a partition that all share the same reduce id, hence + * the given argument. This will not be called twice for the same partition identifier. + * 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 ShuffleWriteSupport#createMapOutputWriter(int, int, long, int)}. + */ + ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException; + + /** + * Commits the writes done by all partition writers returned by all calls to this object's + * {@link #getPartitionWriter(int)}. + *

+ * 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. + *

+ * 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)}. + *

+ * 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; +} diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java similarity index 85% rename from core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java rename to core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index 74c928b0b9c8..af29ef78e056 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import java.io.IOException; import java.io.OutputStream; @@ -24,7 +24,7 @@ /** * :: Experimental :: - * An interface for giving streams / channels for shuffle writes. + * An interface for opening streams to persist partition bytes to a backing data store. * * @since 3.0.0 */ @@ -34,6 +34,8 @@ public interface ShufflePartitionWriter { /** * Opens and returns an underlying {@link OutputStream} that can write bytes to the underlying * data store. + *

+ * The same caller that invokes this method will also close the returned output stream. */ OutputStream openStream() throws IOException; diff --git a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java similarity index 68% rename from core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java rename to core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java index 7ee1d8a55407..6fb7f0fce2f8 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import java.io.IOException; @@ -23,12 +23,21 @@ /** * :: Experimental :: - * An interface for deploying a shuffle map output writer + * A module that returns shuffle writers to persist data that is written by shuffle map tasks. * * @since 3.0.0 */ @Experimental public interface ShuffleWriteSupport { + + /** + * Called once per map task to create a writer that will be responsible for persisting all the + * partitioned bytes written by that map task. + *

+ * The caller of this method will also call either + * {@link ShuffleMapOutputWriter#commitAllPartitions()} upon successful completion of the map + * task, or {@link ShuffleMapOutputWriter#abort(Throwable)} if the map task fails. + */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, diff --git a/core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java b/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java similarity index 98% rename from core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java rename to core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java index 866b61d0bafd..ae8cb36b7e71 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/SupportsTransferTo.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import java.io.IOException; diff --git a/core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java similarity index 91% rename from core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java rename to core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java index 18234d7c4c94..5dd6088d603d 100644 --- a/core/src/main/java/org/apache/spark/api/shuffle/TransferrableWritableByteChannel.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.api.shuffle; +package org.apache.spark.shuffle.api; import java.io.Closeable; import java.io.IOException; @@ -43,6 +43,9 @@ public interface TransferrableWritableByteChannel extends Closeable { /** * Copy all bytes from the source readable byte channel into this byte channel. + *

+ * This method should block until all of the bytes from the source (that is, up until + * numBytesToTransfer) are available in the output storage layer. * * @param source File to transfer bytes from. Do not call anything on this channel other than * {@link FileChannel#transferTo(long, long, WritableByteChannel)}. diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ec828a150235..c5e51ef2cee9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -39,11 +39,11 @@ import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; -import org.apache.spark.api.shuffle.SupportsTransferTo; -import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; -import org.apache.spark.api.shuffle.ShufflePartitionWriter; -import org.apache.spark.api.shuffle.ShuffleWriteSupport; -import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; +import org.apache.spark.shuffle.api.SupportsTransferTo; +import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; +import org.apache.spark.shuffle.api.ShufflePartitionWriter; +import org.apache.spark.shuffle.api.ShuffleWriteSupport; +import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; import org.apache.spark.internal.config.package$; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; @@ -172,14 +172,13 @@ public void write(Iterator> records) throws IOException { partitionLengths = writePartitionedData(mapOutputWriter); mapOutputWriter.commitAllPartitions(); - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), - partitionLengths); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } catch (Exception e) { try { mapOutputWriter.abort(e); } catch (Exception e2) { logger.error("Failed to abort the writer after failing to write map output.", e2); + e.addSuppressed(e2); } throw e; } @@ -211,6 +210,8 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro boolean copyThrewException = true; if (transferToEnabled) { FileInputStream in = new FileInputStream(file); + // Using TransferrableWritableByteChannel to make resource closing consistent between + // this implementation and UnsafeShuffleWriter. TransferrableWritableByteChannel outputChannel = null; try (FileChannel inputChannel = in.getChannel()) { if (writer instanceof SupportsTransferTo) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java index 64ce851e392d..cb8ac86972d3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java @@ -20,12 +20,13 @@ import java.io.IOException; import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; -import org.apache.spark.api.shuffle.TransferrableWritableByteChannel; +import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; +import org.apache.spark.shuffle.api.SupportsTransferTo; import org.apache.spark.util.Utils; /** * This is used when transferTo is enabled but the shuffle plugin hasn't implemented - * {@link org.apache.spark.api.shuffle.SupportsTransferTo}. + * {@link SupportsTransferTo}. *

* This default implementation exists as a convenience to the unsafe shuffle writer and * the bypass merge sort shuffle writers. diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java similarity index 67% rename from core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java rename to core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java index 906600c0f15f..cabcb171ac23 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleDataIO.java @@ -18,19 +18,23 @@ package org.apache.spark.shuffle.sort.io; import org.apache.spark.SparkConf; -import org.apache.spark.api.shuffle.ShuffleExecutorComponents; -import org.apache.spark.api.shuffle.ShuffleDataIO; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; +import org.apache.spark.shuffle.api.ShuffleDataIO; -public class DefaultShuffleDataIO implements ShuffleDataIO { +/** + * Implementation of the {@link ShuffleDataIO} plugin system that replicates the local shuffle + * storage and index file functionality that has historically been used from Spark 2.4 and earlier. + */ +public class LocalDiskShuffleDataIO implements ShuffleDataIO { private final SparkConf sparkConf; - public DefaultShuffleDataIO(SparkConf sparkConf) { + public LocalDiskShuffleDataIO(SparkConf sparkConf) { this.sparkConf = sparkConf; } @Override public ShuffleExecutorComponents executor() { - return new DefaultShuffleExecutorComponents(sparkConf); + return new LocalDiskShuffleExecutorComponents(sparkConf); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java similarity index 83% rename from core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java rename to core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index fb999ad028f0..68d43d0036cb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -19,18 +19,18 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; -import org.apache.spark.api.shuffle.ShuffleExecutorComponents; -import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; +import org.apache.spark.shuffle.api.ShuffleWriteSupport; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.storage.BlockManager; -public class DefaultShuffleExecutorComponents implements ShuffleExecutorComponents { +public class LocalDiskShuffleExecutorComponents implements ShuffleExecutorComponents { private final SparkConf sparkConf; private BlockManager blockManager; private IndexShuffleBlockResolver blockResolver; - public DefaultShuffleExecutorComponents(SparkConf sparkConf) { + public LocalDiskShuffleExecutorComponents(SparkConf sparkConf) { this.sparkConf = sparkConf; } @@ -49,6 +49,6 @@ public ShuffleWriteSupport writes() { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } - return new DefaultShuffleWriteSupport(sparkConf, blockResolver); + return new LocalDiskShuffleWriteSupport(sparkConf, blockResolver); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java similarity index 89% rename from core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java rename to core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 4d2967b251e7..946c9472fc9d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -28,10 +28,10 @@ import org.slf4j.LoggerFactory; import org.apache.spark.SparkConf; -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.shuffle.api.ShuffleMapOutputWriter; +import org.apache.spark.shuffle.api.ShufflePartitionWriter; +import org.apache.spark.shuffle.api.SupportsTransferTo; +import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; import org.apache.spark.internal.config.package$; import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; @@ -39,10 +39,15 @@ import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.util.Utils; -public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { +/** + * Implementation of {@link ShuffleMapOutputWriter} that replicates the functionality of shuffle + * persisting shuffle data to local disk alongside index files, identical to Spark's shuffle + * storage mechanism from Spark 2.4 and earlier. + */ +public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { private static final Logger log = - LoggerFactory.getLogger(DefaultShuffleMapOutputWriter.class); + LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.class); private final int shuffleId; private final int mapId; @@ -60,7 +65,7 @@ public class DefaultShuffleMapOutputWriter implements ShuffleMapOutputWriter { private TimeTrackingOutputStream ts; private BufferedOutputStream outputBufferedFileStream; - public DefaultShuffleMapOutputWriter( + public LocalDiskShuffleMapOutputWriter( int shuffleId, int mapId, int numPartitions, @@ -93,7 +98,7 @@ public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOExcep } else { currChannelPosition = 0L; } - return new DefaultShufflePartitionWriter(partitionId); + return new LocalDiskShufflePartitionWriter(partitionId); } @Override @@ -146,13 +151,13 @@ private void initChannel() throws IOException { } } - private class DefaultShufflePartitionWriter implements SupportsTransferTo { + private class LocalDiskShufflePartitionWriter implements SupportsTransferTo { private final int partitionId; private PartitionWriterStream partStream = null; private PartitionWriterChannel partChannel = null; - private DefaultShufflePartitionWriter(int partitionId) { + private LocalDiskShufflePartitionWriter(int partitionId) { this.partitionId = partitionId; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java similarity index 86% rename from core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java rename to core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java index be392ecbe825..963b0cffe8db 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java @@ -19,16 +19,16 @@ import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; -import org.apache.spark.api.shuffle.ShuffleMapOutputWriter; -import org.apache.spark.api.shuffle.ShuffleWriteSupport; +import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; +import org.apache.spark.shuffle.api.ShuffleWriteSupport; import org.apache.spark.shuffle.IndexShuffleBlockResolver; -public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { +public class LocalDiskShuffleWriteSupport implements ShuffleWriteSupport { private final SparkConf sparkConf; private final IndexShuffleBlockResolver blockResolver; - public DefaultShuffleWriteSupport( + public LocalDiskShuffleWriteSupport( SparkConf sparkConf, IndexShuffleBlockResolver blockResolver) { this.sparkConf = sparkConf; @@ -46,7 +46,7 @@ public ShuffleMapOutputWriter createMapOutputWriter( throw new IllegalStateException( "Task context must be set before creating a map output writer."); } - return new DefaultShuffleMapOutputWriter( + return new LocalDiskShuffleMapOutputWriter( shuffleId, mapId, numPartitions, taskContext.taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3a5d8bb3f838..627fb59d5d50 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -24,7 +24,7 @@ import org.apache.spark.metrics.GarbageCollectionMetrics import org.apache.spark.network.shuffle.Constants import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} -import org.apache.spark.shuffle.sort.io.DefaultShuffleDataIO +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleDataIO import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -799,7 +799,7 @@ package object config { ConfigBuilder("spark.shuffle.io.plugin.class") .doc("Name of the class to use for shuffle IO.") .stringConf - .createWithDefault(classOf[DefaultShuffleDataIO].getName) + .createWithDefault(classOf[LocalDiskShuffleDataIO].getName) private[spark] val SHUFFLE_FILE_BUFFER_SIZE = ConfigBuilder("spark.shuffle.file.buffer") diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index a2cd942849c8..1eadca43b49f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -20,9 +20,11 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap import org.apache.spark._ -import org.apache.spark.api.shuffle.{ShuffleDataIO, ShuffleExecutorComponents} + +import org.apache.spark.api.shuffle.ShuffleExecutorComponents import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ +import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleExecutorComponents} import org.apache.spark.util.Utils /** diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 02811cf67e4d..54bb9f051953 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -32,12 +32,13 @@ import org.scalatest.BeforeAndAfterEach import scala.util.Random import org.apache.spark._ -import org.apache.spark.api.shuffle.ShuffleWriteSupport + import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.shuffle.sort.io.DefaultShuffleWriteSupport +import org.apache.spark.shuffle.api.ShuffleWriteSupport +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleWriteSupport import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -129,7 +130,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte metricsSystem = null, taskMetrics = taskMetrics)) - writeSupport = new DefaultShuffleWriteSupport(conf, blockResolver) + writeSupport = new LocalDiskShuffleWriteSupport(conf, blockResolver) } override def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala similarity index 96% rename from core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala rename to core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index e200eb230468..9add070776bc 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/DefaultShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -30,17 +30,17 @@ import org.mockito.MockitoAnnotations import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach - import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.api.shuffle.SupportsTransferTo + import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver +import org.apache.spark.shuffle.api.SupportsTransferTo import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils -class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { +class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ @Mock(answer = RETURNS_SMART_NULLS) private var shuffleWriteMetrics: ShuffleWriteMetrics = _ @@ -55,7 +55,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft private var tempDir: File = _ private var partitionSizesInMergedFile: Array[Long] = _ private var conf: SparkConf = _ - private var mapOutputWriter: DefaultShuffleMapOutputWriter = _ + private var mapOutputWriter: LocalDiskShuffleMapOutputWriter = _ override def afterEach(): Unit = { try { @@ -90,7 +90,7 @@ class DefaultShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAft } }).when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) - mapOutputWriter = new DefaultShuffleMapOutputWriter( + mapOutputWriter = new LocalDiskShuffleMapOutputWriter( 0, 0, NUM_PARTITIONS, From 594d1e2efa8747673293979521ed80b0e8a3039e Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 8 Jul 2019 16:09:46 -0700 Subject: [PATCH 19/32] Fix style --- .../org/apache/spark/shuffle/sort/SortShuffleManager.scala | 2 -- .../shuffle/sort/BypassMergeSortShuffleWriterSuite.scala | 1 - .../sort/io/LocalDiskShuffleMapOutputWriterSuite.scala | 4 +--- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 1eadca43b49f..763f090e420a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -20,8 +20,6 @@ package org.apache.spark.shuffle.sort import java.util.concurrent.ConcurrentHashMap import org.apache.spark._ - -import org.apache.spark.api.shuffle.ShuffleExecutorComponents import org.apache.spark.internal.{config, Logging} import org.apache.spark.shuffle._ import org.apache.spark.shuffle.api.{ShuffleDataIO, ShuffleExecutorComponents} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 54bb9f051953..fa1fe09ef0e5 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -32,7 +32,6 @@ import org.scalatest.BeforeAndAfterEach import scala.util.Random import org.apache.spark._ - import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 9add070776bc..3624be9d93e3 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.shuffle.sort.io import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} import java.math.BigInteger import java.nio.ByteBuffer -import java.nio.channels.{Channels, WritableByteChannel} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} @@ -30,13 +29,12 @@ import org.mockito.MockitoAnnotations import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.shuffle.api.SupportsTransferTo -import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils From 66aae9152341d0c4ba7e26fcd94b51407906fb29 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 11 Jul 2019 17:42:35 -0700 Subject: [PATCH 20/32] Address comments. --- .../spark/shuffle/api/ShuffleDataIO.java | 6 +-- .../api/ShuffleExecutorComponents.java | 8 +--- .../shuffle/api/ShuffleMapOutputWriter.java | 4 +- .../shuffle/api/ShufflePartitionWriter.java | 12 ++++-- .../shuffle/api/ShuffleWriteSupport.java | 8 +--- .../spark/shuffle/api/SupportsTransferTo.java | 4 +- .../api/TransferrableWritableByteChannel.java | 10 +++-- .../io/LocalDiskShuffleMapOutputWriter.java | 8 +--- ...LocalDiskShuffleMapOutputWriterSuite.scala | 38 +++++++++---------- 9 files changed, 44 insertions(+), 54 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index 72723eef116c..1a4a945dedc2 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -17,11 +17,11 @@ package org.apache.spark.shuffle.api; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: - * An interface for launching Shuffle related components. + * An interface for plugging in modules for storing and reading temporary shuffle data. *

* 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 @@ -32,7 +32,7 @@ * spark.shuffle.io.plugin.class. * @since 3.0.0 */ -@Experimental +@Private public interface ShuffleDataIO { /** diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index dfef914b2949..19749685b7ab 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -17,19 +17,15 @@ package org.apache.spark.shuffle.api; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: * An interface for building shuffle support for Executors. - *

- * At present, submodules returned by this plugin are responsible for managing the - * writing of shuffle bytes (via {@link ShuffleWriteSupport} and the reading of - * shuffle bytes (to be added in subsequent patches). * * @since 3.0.0 */ -@Experimental +@Private public interface ShuffleExecutorComponents { /** diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 744ac22ff356..9159c0006553 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: @@ -28,7 +28,7 @@ * * @since 3.0.0 */ -@Experimental +@Private public interface ShuffleMapOutputWriter { /** diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index af29ef78e056..9cbf2c5034cd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -20,7 +20,7 @@ import java.io.IOException; import java.io.OutputStream; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: @@ -28,19 +28,23 @@ * * @since 3.0.0 */ -@Experimental +@Private public interface ShufflePartitionWriter { /** - * Opens and returns an underlying {@link OutputStream} that can write bytes to the underlying + * Open and return an {@link OutputStream} that can write bytes to the underlying * data store. *

- * The same caller that invokes this method will also close the returned output stream. + * This method will only be called once to write the bytes to the partition. */ OutputStream openStream() throws IOException; /** * Get the number of bytes written by this writer's stream returned by {@link #openStream()}. + *

+ * 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(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java index 6fb7f0fce2f8..f9b9bf65cf83 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: @@ -27,16 +27,12 @@ * * @since 3.0.0 */ -@Experimental +@Private public interface ShuffleWriteSupport { /** * Called once per map task to create a writer that will be responsible for persisting all the * partitioned bytes written by that map task. - *

- * The caller of this method will also call either - * {@link ShuffleMapOutputWriter#commitAllPartitions()} upon successful completion of the map - * task, or {@link ShuffleMapOutputWriter#abort(Throwable)} if the map task fails. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, diff --git a/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java b/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java index ae8cb36b7e71..56c5ea002f24 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: @@ -35,7 +35,7 @@ * * @since 3.0.0 */ -@Experimental +@Private public interface SupportsTransferTo extends ShufflePartitionWriter { /** diff --git a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java index 5dd6088d603d..4039f81b83da 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java @@ -22,7 +22,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; -import org.apache.spark.annotation.Experimental; +import org.apache.spark.annotation.Private; /** * :: Experimental :: @@ -38,7 +38,7 @@ * * @since 3.0.0 */ -@Experimental +@Private public interface TransferrableWritableByteChannel extends Closeable { /** @@ -52,6 +52,8 @@ public interface TransferrableWritableByteChannel extends Closeable { * @param transferStartPosition Start position of the input file to transfer from. * @param numBytesToTransfer Number of bytes to transfer from the given source. */ - void transferFrom(FileChannel source, long transferStartPosition, long numBytesToTransfer) - throws IOException; + void transferFrom( + FileChannel source, + long transferStartPosition, + long numBytesToTransfer) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 946c9472fc9d..4b7e4084b8e3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -109,12 +109,8 @@ public void commitAllPartitions() throws IOException { } @Override - public void abort(Throwable error) { - try { - cleanUp(); - } catch (Exception e) { - log.error("Unable to close appropriate underlying file stream", e); - } + public void abort(Throwable error) throws IOException { + cleanUp(); if (outputTempFile != null && outputTempFile.exists() && !outputTempFile.delete()) { log.warn("Failed to delete temporary shuffle file at {}", outputTempFile.getAbsolutePath()); } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 3624be9d93e3..3c01ec23c7a9 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -26,8 +26,6 @@ import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} import org.mockito.Mock import org.mockito.Mockito.{doAnswer, doNothing, when} import org.mockito.MockitoAnnotations -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} @@ -76,17 +74,15 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA doNothing().when(shuffleWriteMetrics).incWriteTime(anyLong) - doAnswer(new Answer[Void] { - def answer(invocationOnMock: InvocationOnMock): Void = { - partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - mergedOutputFile.delete - tmp.renameTo(mergedOutputFile) - } - null + doAnswer { invocationOnMock => + partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + mergedOutputFile.delete + tmp.renameTo(mergedOutputFile) } - }).when(blockResolver) + null + }.when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) mapOutputWriter = new LocalDiskShuffleMapOutputWriter( 0, @@ -139,15 +135,15 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } test("writing to an outputstream") { - (0 until NUM_PARTITIONS).foreach{ p => + (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) val stream = writer.openStream() - data(p).foreach { i => stream.write(i)} + data(p).foreach { i => stream.write(i) } stream.close() intercept[IllegalStateException] { stream.write(p) } - assert(writer.getNumBytesWritten() == D_LEN) + assert(writer.getNumBytesWritten === D_LEN) } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => D_LEN.toDouble}.toArray @@ -157,7 +153,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } test("writing to a channel") { - (0 until NUM_PARTITIONS).foreach{ p => + (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) @@ -175,7 +171,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA // Bytes require * 4 channel.close() tempFileInput.close() - assert(writer.getNumBytesWritten == D_LEN * 4) + assert(writer.getNumBytesWritten === D_LEN * 4) } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray @@ -185,7 +181,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } test("copyStreams with an outputstream") { - (0 until NUM_PARTITIONS).foreach{ p => + (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) val stream = writer.openStream() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) @@ -195,7 +191,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA Utils.copyStream(in, stream, false, false) in.close() stream.close() - assert(writer.getNumBytesWritten == D_LEN * 4) + assert(writer.getNumBytesWritten === D_LEN * 4) } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray @@ -205,7 +201,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } test("copyStreamsWithNIO with a channel") { - (0 until NUM_PARTITIONS).foreach{ p => + (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) @@ -217,7 +213,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA val in = new FileInputStream(tempFile) channel.transferFrom(in.getChannel, 0L, byteBuffer.remaining()) channel.close() - assert(writer.getNumBytesWritten == D_LEN * 4) + assert(writer.getNumBytesWritten === D_LEN * 4) } mapOutputWriter.commitAllPartitions() val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray From 9f597dd726aba08642c4329534e5ae12ffa6fbe9 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 17 Jul 2019 20:07:13 -0700 Subject: [PATCH 21/32] Address comments. --- .../spark/shuffle/api/ShuffleDataIO.java | 8 ++- .../api/ShuffleExecutorComponents.java | 2 +- .../shuffle/api/ShuffleMapOutputWriter.java | 16 +++--- .../shuffle/api/ShufflePartitionWriter.java | 51 ++++++++++++++++-- .../shuffle/api/ShuffleWriteSupport.java | 2 +- .../spark/shuffle/api/SupportsTransferTo.java | 53 ------------------- .../api/TransferrableWritableByteChannel.java | 2 +- .../sort/BypassMergeSortShuffleWriter.java | 12 +---- ...faultTransferrableWritableByteChannel.java | 10 ++-- .../io/LocalDiskShuffleMapOutputWriter.java | 11 ++-- ...LocalDiskShuffleMapOutputWriterSuite.scala | 7 ++- 11 files changed, 81 insertions(+), 93 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index 1a4a945dedc2..be40cf7f2713 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -20,9 +20,15 @@ import org.apache.spark.annotation.Private; /** - * :: Experimental :: + * :: Private :: * An interface for plugging in modules for storing and reading temporary shuffle data. *

+ * 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. + *

* 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 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 19749685b7ab..d6427ecd1cab 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -20,7 +20,7 @@ import org.apache.spark.annotation.Private; /** - * :: Experimental :: + * :: Private :: * An interface for building shuffle support for Executors. * * @since 3.0.0 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 9159c0006553..ceded4e403bc 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -22,7 +22,7 @@ import org.apache.spark.annotation.Private; /** - * :: Experimental :: + * :: 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. * @@ -32,16 +32,16 @@ public interface ShuffleMapOutputWriter { /** - * Creates a writer that can open an output stream to persist bytes for a given chunk of - * a map task. + * Creates a writer that can open an output stream to persist bytes targeted for a given reduce + * partition id. *

- * The chunk corresponds to bytes in a partition that all share the same reduce id, hence - * the given argument. This will not be called twice for the same partition identifier. - * 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 + * 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 ShuffleWriteSupport#createMapOutputWriter(int, int, long, int)}. */ - ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException; + ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws IOException; /** * Commits the writes done by all partition writers returned by all calls to this object's diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index 9cbf2c5034cd..950f4d0c1b2d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -20,11 +20,16 @@ import java.io.IOException; import java.io.OutputStream; +import java.nio.channels.Channels; import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel; /** - * :: Experimental :: + * :: Private :: * An interface for opening streams to persist partition bytes to a backing data store. + *

+ * This writer stores bytes for one (mapper, reducer) pair, corresponding to one shuffle + * block. * * @since 3.0.0 */ @@ -35,12 +40,52 @@ public interface ShufflePartitionWriter { * Open and return an {@link OutputStream} that can write bytes to the underlying * data store. *

- * This method will only be called once to write the bytes to the partition. + * 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. + *

+ * 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; /** - * Get the number of bytes written by this writer's stream returned by {@link #openStream()}. + * Opens and returns a {@link TransferrableWritableByteChannel} for transferring bytes from + * input byte channels to the underlying shuffle data store. + *

+ * 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. + *

+ * 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 TransferrableWritableByteChannel#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)}. + *

+ * 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. + *

+ * The default implementation should be sufficient for most situations. Only override this + * method if there is a very specific optimization that needs to be built. + */ + default TransferrableWritableByteChannel openTransferrableChannel() throws IOException { + return new DefaultTransferrableWritableByteChannel( + Channels.newChannel(openStream())); + } + + /** + * Returns the number of bytes written either by this writer's output stream opened by + * {@link #openStream()} or the byte channel opened by {@link #openTransferrableChannel()}. *

* 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 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java index f9b9bf65cf83..7c4fb7f10ebf 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java @@ -22,7 +22,7 @@ import org.apache.spark.annotation.Private; /** - * :: Experimental :: + * :: Private :: * A module that returns shuffle writers to persist data that is written by shuffle map tasks. * * @since 3.0.0 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java b/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java deleted file mode 100644 index 56c5ea002f24..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/api/SupportsTransferTo.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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; - -/** - * :: Experimental :: - * Indicates that partition writers can transfer bytes directly from input byte channels to - * output channels that stream data to the underlying shuffle partition storage medium. - *

- * This API is separated out for advanced users because it only needs to be used for - * specific low-level optimizations. The idea is that the returned channel can transfer bytes - * from the input file channel out to the backing storage system without copying data into - * memory. - *

- * Most shuffle plugin implementations should use {@link ShufflePartitionWriter} instead. - * - * @since 3.0.0 - */ -@Private -public interface SupportsTransferTo extends ShufflePartitionWriter { - - /** - * Opens and returns a {@link TransferrableWritableByteChannel} for transferring bytes from - * input byte channels to the underlying shuffle data store. - */ - TransferrableWritableByteChannel openTransferrableChannel() throws IOException; - - /** - * Returns the number of bytes written either by this writer's output stream opened by - * {@link #openStream()} or the byte channel opened by {@link #openTransferrableChannel()}. - */ - @Override - long getNumBytesWritten(); -} diff --git a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java index 4039f81b83da..1e7c0246a13d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java @@ -25,7 +25,7 @@ import org.apache.spark.annotation.Private; /** - * :: Experimental :: + * :: Private :: * Represents an output byte channel that can copy bytes from input file channels to some * arbitrary storage system. *

diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index c5e51ef2cee9..ea0c2f928646 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -21,7 +21,6 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.OutputStream; -import java.nio.channels.Channels; import java.nio.channels.FileChannel; import javax.annotation.Nullable; @@ -39,7 +38,6 @@ import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; -import org.apache.spark.shuffle.api.SupportsTransferTo; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.ShuffleWriteSupport; @@ -212,16 +210,8 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro FileInputStream in = new FileInputStream(file); // Using TransferrableWritableByteChannel to make resource closing consistent between // this implementation and UnsafeShuffleWriter. - TransferrableWritableByteChannel outputChannel = null; + TransferrableWritableByteChannel outputChannel = writer.openTransferrableChannel(); try (FileChannel inputChannel = in.getChannel()) { - if (writer instanceof SupportsTransferTo) { - outputChannel = ((SupportsTransferTo) writer).openTransferrableChannel(); - } else { - // Use default transferrable writable channel anyways in order to have parity with - // UnsafeShuffleWriter. - outputChannel = new DefaultTransferrableWritableByteChannel( - Channels.newChannel(writer.openStream())); - } outputChannel.transferFrom(inputChannel, 0L, inputChannel.size()); copyThrewException = false; } finally { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java index cb8ac86972d3..ee302029743d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java @@ -20,13 +20,13 @@ import java.io.IOException; import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; +import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; -import org.apache.spark.shuffle.api.SupportsTransferTo; import org.apache.spark.util.Utils; /** - * This is used when transferTo is enabled but the shuffle plugin hasn't implemented - * {@link SupportsTransferTo}. + * This is used when transferTo is enabled but the shuffle plugin hasn't overridden the + * implementation of {@link ShufflePartitionWriter#openTransferrableChannel()}. *

* This default implementation exists as a convenience to the unsafe shuffle writer and * the bypass merge sort shuffle writers. @@ -41,7 +41,9 @@ public DefaultTransferrableWritableByteChannel(WritableByteChannel delegate) { @Override public void transferFrom( - FileChannel source, long transferStartPosition, long numBytesToTransfer) { + FileChannel source, + long transferStartPosition, + long numBytesToTransfer) { Utils.copyFileStreamNIO(source, delegate, transferStartPosition, numBytesToTransfer); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 4b7e4084b8e3..6108fe36ab7f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -30,7 +30,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; -import org.apache.spark.shuffle.api.SupportsTransferTo; import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; import org.apache.spark.internal.config.package$; import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel; @@ -85,11 +84,11 @@ public LocalDiskShuffleMapOutputWriter( } @Override - public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOException { - if (partitionId <= lastPartitionId) { + public ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws IOException { + if (reducePartitionId <= lastPartitionId) { throw new IllegalArgumentException("Partitions should be requested in increasing order."); } - lastPartitionId = partitionId; + lastPartitionId = reducePartitionId; if (outputTempFile == null) { outputTempFile = Utils.tempFileWith(outputFile); } @@ -98,7 +97,7 @@ public ShufflePartitionWriter getPartitionWriter(int partitionId) throws IOExcep } else { currChannelPosition = 0L; } - return new LocalDiskShufflePartitionWriter(partitionId); + return new LocalDiskShufflePartitionWriter(reducePartitionId); } @Override @@ -147,7 +146,7 @@ private void initChannel() throws IOException { } } - private class LocalDiskShufflePartitionWriter implements SupportsTransferTo { + private class LocalDiskShufflePartitionWriter implements ShufflePartitionWriter { private final int partitionId; private PartitionWriterStream partStream = null; diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 3c01ec23c7a9..2718a8098c76 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -32,7 +32,6 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.shuffle.api.SupportsTransferTo import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils @@ -155,7 +154,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA test("writing to a channel") { (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() + val channel = writer.openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() intBuffer.put(data(p)) @@ -174,7 +173,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA assert(writer.getNumBytesWritten === D_LEN * 4) } mapOutputWriter.commitAllPartitions() - val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray + val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble }.toArray assert(partitionSizesInMergedFile === partitionLengths) assert(mergedOutputFile.length() === partitionLengths.sum) assert(data === readRecordsFromFile(true)) @@ -203,7 +202,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA test("copyStreamsWithNIO with a channel") { (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.asInstanceOf[SupportsTransferTo].openTransferrableChannel() + val channel = writer.openTransferrableChannel() val byteBuffer = ByteBuffer.allocate(D_LEN * 4) val intBuffer = byteBuffer.asIntBuffer() intBuffer.put(data(p)) From 86c182937df6f085d76bfb6c626aec4be0556632 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 18 Jul 2019 15:49:08 -0700 Subject: [PATCH 22/32] Restructure test --- .../sort/BypassMergeSortShuffleWriter.java | 22 ++- .../scala/org/apache/spark/ShuffleSuite.scala | 6 +- .../BypassMergeSortShuffleWriterSuite.scala | 111 ++++++------- ...LocalDiskShuffleMapOutputWriterSuite.scala | 155 ++++++------------ 4 files changed, 110 insertions(+), 184 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index ea0c2f928646..91b51c5794b1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -210,24 +210,30 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro FileInputStream in = new FileInputStream(file); // Using TransferrableWritableByteChannel to make resource closing consistent between // this implementation and UnsafeShuffleWriter. - TransferrableWritableByteChannel outputChannel = writer.openTransferrableChannel(); - try (FileChannel inputChannel = in.getChannel()) { - outputChannel.transferFrom(inputChannel, 0L, inputChannel.size()); - copyThrewException = false; + try { + TransferrableWritableByteChannel outputChannel = writer.openTransferrableChannel(); + try (FileChannel inputChannel = in.getChannel()) { + outputChannel.transferFrom(inputChannel, 0L, inputChannel.size()); + copyThrewException = false; + } finally { + Closeables.close(outputChannel, copyThrewException); + } } finally { Closeables.close(in, copyThrewException); - Closeables.close(outputChannel, copyThrewException); } } else { FileInputStream in = new FileInputStream(file); OutputStream outputStream = null; try { outputStream = writer.openStream(); - Utils.copyStream(in, outputStream, false, false); - copyThrewException = false; + try { + Utils.copyStream(in, outputStream, false, false); + copyThrewException = false; + } finally { + Closeables.close(outputStream, copyThrewException); + } } finally { Closeables.close(in, copyThrewException); - Closeables.close(outputStream, copyThrewException); } } if (!file.delete()) { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 33ee6e8a21ae..881f6e5555b8 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -383,9 +383,9 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // simultaneously, and everything is still OK def writeAndClose( - writer: ShuffleWriter[Int, Int], - taskContext: TaskContext)( - iter: Iterator[(Int, Int)]): Option[MapStatus] = { + writer: ShuffleWriter[Int, Int], + taskContext: TaskContext)( + iter: Iterator[(Int, Int)]): Option[MapStatus] = { TaskContext.setTaskContext(taskContext) try { val files = writer.write(iter) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index fa1fe09ef0e5..547dfe608f4d 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -22,6 +22,7 @@ import java.util.{Properties, UUID} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.Random import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS @@ -29,7 +30,6 @@ import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach -import scala.util.Random import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} @@ -83,14 +83,10 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte null }.when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) - when(blockManager.getDiskWriter( - any[BlockId], - any[File], - any[SerializerInstance], - anyInt(), - any[ShuffleWriteMetrics] - )).thenAnswer((invocation: InvocationOnMock) => { + + doAnswer((invocation: InvocationOnMock) => { val args = invocation.getArguments val manager = new SerializerManager(new JavaSerializer(conf), conf) new DiskBlockObjectWriter( @@ -102,17 +98,25 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte args(4).asInstanceOf[ShuffleWriteMetrics], blockId = args(0).asInstanceOf[BlockId] ) - }) - when(diskBlockManager.createTempShuffleBlock()).thenAnswer((_: InvocationOnMock) => { + }).when(blockManager) + .getDiskWriter( + any[BlockId], + any[File], + any[SerializerInstance], + anyInt(), + any[ShuffleWriteMetrics]) + + doAnswer((_: InvocationOnMock) => { val blockId = new TempShuffleBlockId(UUID.randomUUID) val file = new File(tempDir, blockId.name) blockIdToFileMap.put(blockId, file) temporaryFilesCreated += file (blockId, file) - }) - when(diskBlockManager.getFile(any[BlockId])).thenAnswer { (invocation: InvocationOnMock) => + }).when(diskBlockManager).createTempShuffleBlock() + + doAnswer((invocation: InvocationOnMock) => { blockIdToFileMap(invocation.getArguments.head.asInstanceOf[BlockId]) - } + }).when(diskBlockManager).getFile(any[BlockId]) val memoryManager = new TestMemoryManager(conf) val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) @@ -148,11 +152,11 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockManager, shuffleHandle, 0, // MapId - 0L, + 0L, // MapTaskAttemptId conf, taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) + writeSupport) + writer.write(Iterator.empty) writer.stop( /* success = */ true) assert(writer.getPartitionLengths.sum === 0) @@ -166,55 +170,32 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte assert(taskMetrics.memoryBytesSpilled === 0) } - test("write with some empty partitions") { - val transferConf = conf.clone.set("spark.file.transferTo", "false") - def records: Iterator[(Int, Int)] = - Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) - val writer = new BypassMergeSortShuffleWriter[Int, Int]( - blockManager, - shuffleHandle, - 0, // MapId - 0L, - transferConf, - taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) - writer.write(records) - writer.stop( /* success = */ true) - assert(temporaryFilesCreated.nonEmpty) - assert(writer.getPartitionLengths.sum === outputFile.length()) - assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files - assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted - val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics - assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) - assert(shuffleWriteMetrics.recordsWritten === records.length) - assert(taskMetrics.diskBytesSpilled === 0) - assert(taskMetrics.memoryBytesSpilled === 0) - } - - test("write with some empty partitions with transferTo") { - def records: Iterator[(Int, Int)] = - Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) - val writer = new BypassMergeSortShuffleWriter[Int, Int]( - blockManager, - shuffleHandle, - 0, // MapId - 0L, - conf, - taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) - writer.write(records) - writer.stop( /* success = */ true) - assert(temporaryFilesCreated.nonEmpty) - assert(writer.getPartitionLengths.sum === outputFile.length()) - assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files - assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temporary files were deleted - val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics - assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) - assert(shuffleWriteMetrics.recordsWritten === records.length) - assert(taskMetrics.diskBytesSpilled === 0) - assert(taskMetrics.memoryBytesSpilled === 0) + Seq(true, false).foreach { transferTo => + test(s"write with some empty partitions - transferTo $transferTo") { + val transferConf = conf.clone.set("spark.file.transferTo", transferTo.toString) + def records: Iterator[(Int, Int)] = + Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + val writer = new BypassMergeSortShuffleWriter[Int, Int]( + blockManager, + shuffleHandle, + 0, // MapId + 0L, + transferConf, + taskContext.taskMetrics().shuffleWriteMetrics, + writeSupport + ) + writer.write(records) + writer.stop( /* success = */ true) + assert(temporaryFilesCreated.nonEmpty) + assert(writer.getPartitionLengths.sum === outputFile.length()) + assert(writer.getPartitionLengths.count(_ == 0L) === 4) // should be 4 zero length files + assert(temporaryFilesCreated.count(_.exists()) === 0) // check that temp files were deleted + val shuffleWriteMetrics = taskContext.taskMetrics().shuffleWriteMetrics + assert(shuffleWriteMetrics.bytesWritten === outputFile.length()) + assert(shuffleWriteMetrics.recordsWritten === records.length) + assert(taskMetrics.diskBytesSpilled === 0) + assert(taskMetrics.memoryBytesSpilled === 0) + } } test("only generate temp shuffle file for non-empty partition") { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 2718a8098c76..b391072bf467 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.shuffle.sort.io -import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} -import java.math.BigInteger -import java.nio.ByteBuffer +import java.io.{ByteArrayOutputStream, File, FileInputStream, FileOutputStream} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} @@ -32,18 +30,28 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.Utils class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndAfterEach { - @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) private var shuffleWriteMetrics: ShuffleWriteMetrics = _ + @Mock(answer = RETURNS_SMART_NULLS) + private var blockResolver: IndexShuffleBlockResolver = _ + + @Mock(answer = RETURNS_SMART_NULLS) + private var shuffleWriteMetrics: ShuffleWriteMetrics = _ private val NUM_PARTITIONS = 4 - private val D_LEN = 10 - private val data: Array[Array[Int]] = (0 until NUM_PARTITIONS).map { - p => (1 to D_LEN).map(_ + p).toArray }.toArray + private val data: Array[Array[Byte]] = (0 until NUM_PARTITIONS).map { + p => { + if (p == 3) { + Array.emptyByteArray + } else { + (0 to p * 10).map(_ + p).map(_.toByte).toArray + } + } + }.toArray + + private val partitionLengths = data.map(_.length) private var tempFile: File = _ private var mergedOutputFile: File = _ @@ -92,47 +100,6 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA conf) } - private def readRecordsFromFile(fromByte: Boolean): Array[Array[Int]] = { - var startOffset = 0L - val result = new Array[Array[Int]](NUM_PARTITIONS) - (0 until NUM_PARTITIONS).foreach { p => - val partitionSize = partitionSizesInMergedFile(p).toInt - lazy val inner = new Array[Int](partitionSize) - lazy val innerBytebuffer = ByteBuffer.allocate(partitionSize) - if (partitionSize > 0) { - val in = new FileInputStream(mergedOutputFile) - in.getChannel.position(startOffset) - val lin = new LimitedInputStream(in, partitionSize) - var nonEmpty = true - var count = 0 - while (nonEmpty) { - try { - val readBit = lin.read() - if (fromByte) { - innerBytebuffer.put(readBit.toByte) - } else { - inner(count) = readBit - } - count += 1 - } catch { - case _: Exception => - nonEmpty = false - } - } - in.close() - } - if (fromByte) { - result(p) = innerBytebuffer.array().sliding(4, 4).map { b => - new BigInteger(b).intValue() - }.toArray - } else { - result(p) = inner - } - startOffset += partitionSize - } - result - } - test("writing to an outputstream") { (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) @@ -142,82 +109,54 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA intercept[IllegalStateException] { stream.write(p) } - assert(writer.getNumBytesWritten === D_LEN) + assert(writer.getNumBytesWritten === data(p).length) } - mapOutputWriter.commitAllPartitions() - val partitionLengths = (0 until NUM_PARTITIONS).map { _ => D_LEN.toDouble}.toArray - assert(partitionSizesInMergedFile === partitionLengths) - assert(mergedOutputFile.length() === partitionLengths.sum) - assert(data === readRecordsFromFile(false)) + verifyWrittenRecords() } test("writing to a channel") { (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.openTransferrableChannel() - val byteBuffer = ByteBuffer.allocate(D_LEN * 4) - val intBuffer = byteBuffer.asIntBuffer() - intBuffer.put(data(p)) - val numBytes = byteBuffer.remaining() val outputTempFile = File.createTempFile("channelTemp", "", tempDir) val outputTempFileStream = new FileOutputStream(outputTempFile) - Utils.copyStream( - new ByteBufferInputStream(byteBuffer), - outputTempFileStream, - closeStreams = true) + outputTempFileStream.write(data(p)) + outputTempFileStream.close() val tempFileInput = new FileInputStream(outputTempFile) - channel.transferFrom(tempFileInput.getChannel, 0L, numBytes) - // Bytes require * 4 - channel.close() - tempFileInput.close() - assert(writer.getNumBytesWritten === D_LEN * 4) + val channel = writer.openTransferrableChannel() + Utils.tryWithResource(new FileInputStream(outputTempFile)) { tempFileInput => + Utils.tryWithResource(writer.openTransferrableChannel()) { channel => + channel.transferFrom(tempFileInput.getChannel, 0L, data(p).length) + } + } + assert(writer.getNumBytesWritten === data(p).length) } - mapOutputWriter.commitAllPartitions() - val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble }.toArray - assert(partitionSizesInMergedFile === partitionLengths) - assert(mergedOutputFile.length() === partitionLengths.sum) - assert(data === readRecordsFromFile(true)) + verifyWrittenRecords() } - test("copyStreams with an outputstream") { + private def readRecordsFromFile() = { + var startOffset = 0L + val result = new Array[Array[Byte]](NUM_PARTITIONS) (0 until NUM_PARTITIONS).foreach { p => - val writer = mapOutputWriter.getPartitionWriter(p) - val stream = writer.openStream() - val byteBuffer = ByteBuffer.allocate(D_LEN * 4) - val intBuffer = byteBuffer.asIntBuffer() - intBuffer.put(data(p)) - val in = new ByteArrayInputStream(byteBuffer.array()) - Utils.copyStream(in, stream, false, false) - in.close() - stream.close() - assert(writer.getNumBytesWritten === D_LEN * 4) + val partitionSize = data(p).length + if (partitionSize > 0) { + val in = new FileInputStream(mergedOutputFile) + in.getChannel.position(startOffset) + val lin = new LimitedInputStream(in, partitionSize) + val bytesOut = new ByteArrayOutputStream + Utils.copyStream(lin, bytesOut, true, true) + result(p) = bytesOut.toByteArray + } else { + result(p) = Array.emptyByteArray + } + startOffset += partitionSize } - mapOutputWriter.commitAllPartitions() - val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray - assert(partitionSizesInMergedFile === partitionLengths) - assert(mergedOutputFile.length() === partitionLengths.sum) - assert(data === readRecordsFromFile(true)) + result } - test("copyStreamsWithNIO with a channel") { - (0 until NUM_PARTITIONS).foreach { p => - val writer = mapOutputWriter.getPartitionWriter(p) - val channel = writer.openTransferrableChannel() - val byteBuffer = ByteBuffer.allocate(D_LEN * 4) - val intBuffer = byteBuffer.asIntBuffer() - intBuffer.put(data(p)) - val out = new FileOutputStream(tempFile) - out.write(byteBuffer.array()) - out.close() - val in = new FileInputStream(tempFile) - channel.transferFrom(in.getChannel, 0L, byteBuffer.remaining()) - channel.close() - assert(writer.getNumBytesWritten === D_LEN * 4) - } + private def verifyWrittenRecords(): Unit = { mapOutputWriter.commitAllPartitions() - val partitionLengths = (0 until NUM_PARTITIONS).map { _ => (D_LEN * 4).toDouble}.toArray assert(partitionSizesInMergedFile === partitionLengths) assert(mergedOutputFile.length() === partitionLengths.sum) - assert(data === readRecordsFromFile(true)) + assert(data === readRecordsFromFile()) } } From a7885ae733f1779b8ed7ac282befe7b831f6cd42 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 19 Jul 2019 11:41:56 -0700 Subject: [PATCH 23/32] Add ShuffleWriteMetricsReporter to the createMapOutputWriter API. --- .../shuffle/api/ShuffleMapOutputWriter.java | 4 +++- .../shuffle/api/ShuffleWriteSupport.java | 23 +++++++++++++++---- .../sort/BypassMergeSortShuffleWriter.java | 4 ++-- .../sort/io/LocalDiskShuffleWriteSupport.java | 13 ++++------- .../scala/org/apache/spark/ShuffleSuite.scala | 2 -- .../BypassMergeSortShuffleWriterSuite.scala | 11 --------- 6 files changed, 28 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index ceded4e403bc..17e553c154c3 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * :: Private :: @@ -39,7 +40,8 @@ public interface ShuffleMapOutputWriter { * 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 ShuffleWriteSupport#createMapOutputWriter(int, int, long, int)}. + * {@link ShuffleWriteSupport#createMapOutputWriter( + * int, int, long, int, ShuffleWriteMetricsReporter)}. */ ShufflePartitionWriter getPartitionWriter(int reducePartitionId) throws IOException; diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java index 7c4fb7f10ebf..85d13ac64e45 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * :: Private :: @@ -33,10 +34,24 @@ public interface ShuffleWriteSupport { /** * 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 stage of the map task + * @param mapId Within the shuffle stage, 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. + * @param mapTaskWriteMetrics The map task's write metrics, which can be updated by the returned + * writer. The updates that are posted to this reporter are listed in + * the Spark UI. Note that the caller will update the total write time + * at the end of the map task, so implementations should not call + * {@link ShuffleWriteMetricsReporter#incWriteTime(long)}. */ ShuffleMapOutputWriter createMapOutputWriter( - int shuffleId, - int mapId, - long mapTaskAttemptId, - int numPartitions) throws IOException; + int shuffleId, + int mapId, + long mapTaskAttemptId, + int numPartitions, + ShuffleWriteMetricsReporter mapTaskWriteMetrics) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 91b51c5794b1..8860b94505cd 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -129,7 +129,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleWriteSupport - .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); + .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions, writeMetrics); try { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; @@ -223,7 +223,7 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } } else { FileInputStream in = new FileInputStream(file); - OutputStream outputStream = null; + OutputStream outputStream; try { outputStream = writer.openStream(); try { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java index 963b0cffe8db..ddb697338cc8 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java @@ -18,7 +18,7 @@ package org.apache.spark.shuffle.sort.io; import org.apache.spark.SparkConf; -import org.apache.spark.TaskContext; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShuffleWriteSupport; import org.apache.spark.shuffle.IndexShuffleBlockResolver; @@ -40,14 +40,9 @@ public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, long mapTaskAttemptId, - int numPartitions) { - TaskContext taskContext = TaskContext.get(); - if (taskContext == null) { - throw new IllegalStateException( - "Task context must be set before creating a map output writer."); - } + int numPartitions, + ShuffleWriteMetricsReporter writeMetrics) { return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, - taskContext.taskMetrics().shuffleWriteMetrics(), blockResolver, sparkConf); + shuffleId, mapId, numPartitions, writeMetrics, blockResolver, sparkConf); } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 881f6e5555b8..923c9c90447f 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -386,7 +386,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC writer: ShuffleWriter[Int, Int], taskContext: TaskContext)( iter: Iterator[(Int, Int)]): Option[MapStatus] = { - TaskContext.setTaskContext(taskContext) try { val files = writer.write(iter) writer.stop(true) @@ -411,7 +410,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val taskContext = new TaskContextImpl( 1, 0, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem) - TaskContext.setTaskContext(taskContext) val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, taskContext, metrics) TaskContext.unset() diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 547dfe608f4d..d107bdde7d75 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -122,17 +122,6 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) - TaskContext.setTaskContext(new TaskContextImpl( - stageId = 0, - stageAttemptNumber = 0, - partitionId = 0, - taskAttemptId = Random.nextInt(10000), - attemptNumber = 0, - taskMemoryManager = taskMemoryManager, - localProperties = new Properties, - metricsSystem = null, - taskMetrics = taskMetrics)) - writeSupport = new LocalDiskShuffleWriteSupport(conf, blockResolver) } From 9893c6c7762bf59f0940f857141325f5f61e83cf Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 19 Jul 2019 12:06:00 -0700 Subject: [PATCH 24/32] Add more documentation --- .../apache/spark/shuffle/api/ShuffleMapOutputWriter.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 17e553c154c3..2341bb4d2939 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -42,6 +42,12 @@ public interface ShuffleMapOutputWriter { * provided upon the creation of this map output writer via * {@link ShuffleWriteSupport#createMapOutputWriter( * int, int, long, int, ShuffleWriteMetricsReporter)}. + *

+ * 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; From cd897e7fd9e82dee6299251d522e6397b10bd84d Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 19 Jul 2019 14:11:32 -0700 Subject: [PATCH 25/32] REfactor reading records from file in test --- ...LocalDiskShuffleMapOutputWriterSuite.scala | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index b391072bf467..334b8be4fe7b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.shuffle.sort.io -import java.io.{ByteArrayOutputStream, File, FileInputStream, FileOutputStream} +import java.io.{File, FileInputStream, FileOutputStream} +import java.nio.file.Files +import java.util.Arrays import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} @@ -28,7 +30,6 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.util.Utils @@ -134,22 +135,12 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA } private def readRecordsFromFile() = { - var startOffset = 0L - val result = new Array[Array[Byte]](NUM_PARTITIONS) - (0 until NUM_PARTITIONS).foreach { p => - val partitionSize = data(p).length - if (partitionSize > 0) { - val in = new FileInputStream(mergedOutputFile) - in.getChannel.position(startOffset) - val lin = new LimitedInputStream(in, partitionSize) - val bytesOut = new ByteArrayOutputStream - Utils.copyStream(lin, bytesOut, true, true) - result(p) = bytesOut.toByteArray - } else { - result(p) = Array.emptyByteArray - } - startOffset += partitionSize - } + val mergedOutputBytes = Files.readAllBytes(mergedOutputFile.toPath) + val result = (0 until NUM_PARTITIONS).map { part => + val startOffset = data.slice(0, part).map(_.length).sum + val partitionSize = data(part).length + Arrays.copyOfRange(mergedOutputBytes, startOffset, startOffset + partitionSize) + }.toArray result } From 9f17b9bbf0d3d5677abf424c5b2d4d3b93dfc95a Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 24 Jul 2019 16:04:26 -0700 Subject: [PATCH 26/32] Address comments --- .../spark/shuffle/api/ShuffleDataIO.java | 3 - .../api/ShuffleExecutorComponents.java | 29 +++++++-- .../shuffle/api/ShufflePartitionWriter.java | 15 +++-- .../shuffle/api/ShuffleWriteSupport.java | 4 +- .../api/TransferrableWritableByteChannel.java | 59 ------------------- .../api/WritableByteChannelWrapper.java | 42 +++++++++++++ .../sort/BypassMergeSortShuffleWriter.java | 19 +++--- ...faultTransferrableWritableByteChannel.java | 54 ----------------- .../io/DefaultWritableByteChannelWrapper.java | 49 +++++++++++++++ .../LocalDiskShuffleExecutorComponents.java | 27 +++++++-- .../io/LocalDiskShuffleMapOutputWriter.java | 18 +++--- .../sort/io/LocalDiskShuffleWriteSupport.java | 48 --------------- .../spark/internal/config/package.scala | 2 +- .../shuffle/sort/SortShuffleManager.scala | 2 +- .../BypassMergeSortShuffleWriterSuite.scala | 24 ++++---- ...LocalDiskShuffleMapOutputWriterSuite.scala | 53 +++++++++-------- 16 files changed, 206 insertions(+), 242 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/api/WritableByteChannelWrapper.java delete mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java create mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java delete mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index be40cf7f2713..4b5501f4dc63 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -44,9 +44,6 @@ public interface ShuffleDataIO { /** * Called once on executor processes to bootstrap the shuffle data storage modules that * are only invoked on the executors. - *

- * At this point, this module is responsible for reading and writing shuffle data bytes - * from the backing store. */ ShuffleExecutorComponents executor(); } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index d6427ecd1cab..12be3b2dd0a9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -17,7 +17,9 @@ package org.apache.spark.shuffle.api; +import java.io.IOException; import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * :: Private :: @@ -35,11 +37,26 @@ public interface ShuffleExecutorComponents { void initializeExecutor(String appId, String execId); /** - * Returns the modules that are responsible for persisting shuffle data to the backing - * store. - *

- * This may be called multiple times on each executor. Implementations should not make - * any assumptions about the lifetime of the returned module. + * 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. + * @param mapTaskWriteMetrics The map task's write metrics, which can be updated by the returned + * writer. The updates that are posted to this reporter are listed in + * the Spark UI. Note that the caller will update the total write time + * at the end of the map task, so implementations should not call + * {@link ShuffleWriteMetricsReporter#incWriteTime(long)}. */ - ShuffleWriteSupport writes(); + ShuffleMapOutputWriter createMapOutputWriter( + int shuffleId, + int mapId, + long mapTaskAttemptId, + int numPartitions, + ShuffleWriteMetricsReporter mapTaskWriteMetrics) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index 950f4d0c1b2d..8a94dfc1c7aa 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -19,10 +19,10 @@ import java.io.IOException; import java.io.OutputStream; - import java.nio.channels.Channels; + import org.apache.spark.annotation.Private; -import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel; +import org.apache.spark.shuffle.sort.io.DefaultWritableByteChannelWrapper; /** * :: Private :: @@ -56,7 +56,7 @@ public interface ShufflePartitionWriter { OutputStream openStream() throws IOException; /** - * Opens and returns a {@link TransferrableWritableByteChannel} for transferring bytes from + * Opens and returns a {@link WritableByteChannelWrapper} for transferring bytes from * input byte channels to the underlying shuffle data store. *

* This method will only be called once on this partition writer in the map task, to write the @@ -67,7 +67,7 @@ public interface ShufflePartitionWriter { * 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 TransferrableWritableByteChannel#close()} does not close the resource, since it + * {@link WritableByteChannelWrapper#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)}. @@ -78,14 +78,13 @@ public interface ShufflePartitionWriter { * The default implementation should be sufficient for most situations. Only override this * method if there is a very specific optimization that needs to be built. */ - default TransferrableWritableByteChannel openTransferrableChannel() throws IOException { - return new DefaultTransferrableWritableByteChannel( - Channels.newChannel(openStream())); + default WritableByteChannelWrapper openChannelWrapper() throws IOException { + return new DefaultWritableByteChannelWrapper(Channels.newChannel(openStream())); } /** * Returns the number of bytes written either by this writer's output stream opened by - * {@link #openStream()} or the byte channel opened by {@link #openTransferrableChannel()}. + * {@link #openStream()} or the byte channel opened by {@link #openChannelWrapper()}. *

* 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 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java index 85d13ac64e45..cdf06ce715ca 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java @@ -35,8 +35,8 @@ public interface ShuffleWriteSupport { * 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 stage of the map task - * @param mapId Within the shuffle stage, the identifier of the 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. diff --git a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java deleted file mode 100644 index 1e7c0246a13d..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/api/TransferrableWritableByteChannel.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.io.IOException; - -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; -import org.apache.spark.annotation.Private; - -/** - * :: Private :: - * Represents an output byte channel that can copy bytes from input file channels to some - * arbitrary storage system. - *

- * This API is provided for advanced users who can transfer bytes from a file channel to - * some output sink without copying data into memory. Most users should not need to use - * this functionality; this is primarily provided for the built-in shuffle storage backends - * that persist shuffle files on local disk. - *

- * For a simpler alternative, see {@link ShufflePartitionWriter}. - * - * @since 3.0.0 - */ -@Private -public interface TransferrableWritableByteChannel extends Closeable { - - /** - * Copy all bytes from the source readable byte channel into this byte channel. - *

- * This method should block until all of the bytes from the source (that is, up until - * numBytesToTransfer) are available in the output storage layer. - * - * @param source File to transfer bytes from. Do not call anything on this channel other than - * {@link FileChannel#transferTo(long, long, WritableByteChannel)}. - * @param transferStartPosition Start position of the input file to transfer from. - * @param numBytesToTransfer Number of bytes to transfer from the given source. - */ - void transferFrom( - FileChannel source, - long transferStartPosition, - long numBytesToTransfer) throws IOException; -} diff --git a/core/src/main/java/org/apache/spark/shuffle/api/WritableByteChannelWrapper.java b/core/src/main/java/org/apache/spark/shuffle/api/WritableByteChannelWrapper.java new file mode 100644 index 000000000000..a204903008a5 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/api/WritableByteChannelWrapper.java @@ -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}. + *

+ * 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(); +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 8860b94505cd..eed62585df4d 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -38,10 +38,10 @@ import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; +import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; -import org.apache.spark.shuffle.api.ShuffleWriteSupport; -import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; +import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.internal.config.package$; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; @@ -87,7 +87,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int mapId; private final long mapTaskAttemptId; private final Serializer serializer; - private final ShuffleWriteSupport shuffleWriteSupport; + private final ShuffleExecutorComponents shuffleExecutorComponents; /** Array of file writers, one for each partition */ private DiskBlockObjectWriter[] partitionWriters; @@ -109,7 +109,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { long mapTaskAttemptId, SparkConf conf, ShuffleWriteMetricsReporter writeMetrics, - ShuffleWriteSupport shuffleWriteSupport) { + ShuffleExecutorComponents shuffleExecutorComponents) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); @@ -122,13 +122,13 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.numPartitions = partitioner.numPartitions(); this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); - this.shuffleWriteSupport = shuffleWriteSupport; + this.shuffleExecutorComponents = shuffleExecutorComponents; } @Override public void write(Iterator> records) throws IOException { assert (partitionWriters == null); - ShuffleMapOutputWriter mapOutputWriter = shuffleWriteSupport + ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions, writeMetrics); try { if (!records.hasNext()) { @@ -208,12 +208,13 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro boolean copyThrewException = true; if (transferToEnabled) { FileInputStream in = new FileInputStream(file); - // Using TransferrableWritableByteChannel to make resource closing consistent between + // Using WritableByteChannelWrapper to make resource closing consistent between // this implementation and UnsafeShuffleWriter. try { - TransferrableWritableByteChannel outputChannel = writer.openTransferrableChannel(); + WritableByteChannelWrapper outputChannel = writer.openChannelWrapper(); try (FileChannel inputChannel = in.getChannel()) { - outputChannel.transferFrom(inputChannel, 0L, inputChannel.size()); + Utils.copyFileStreamNIO( + inputChannel, outputChannel.channel(), 0L, inputChannel.size()); copyThrewException = false; } finally { Closeables.close(outputChannel, copyThrewException); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java b/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java deleted file mode 100644 index ee302029743d..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/sort/DefaultTransferrableWritableByteChannel.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * 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.sort; - -import java.io.IOException; -import java.nio.channels.FileChannel; -import java.nio.channels.WritableByteChannel; -import org.apache.spark.shuffle.api.ShufflePartitionWriter; -import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; -import org.apache.spark.util.Utils; - -/** - * This is used when transferTo is enabled but the shuffle plugin hasn't overridden the - * implementation of {@link ShufflePartitionWriter#openTransferrableChannel()}. - *

- * This default implementation exists as a convenience to the unsafe shuffle writer and - * the bypass merge sort shuffle writers. - */ -public class DefaultTransferrableWritableByteChannel implements TransferrableWritableByteChannel { - - private final WritableByteChannel delegate; - - public DefaultTransferrableWritableByteChannel(WritableByteChannel delegate) { - this.delegate = delegate; - } - - @Override - public void transferFrom( - FileChannel source, - long transferStartPosition, - long numBytesToTransfer) { - Utils.copyFileStreamNIO(source, delegate, transferStartPosition, numBytesToTransfer); - } - - @Override - public void close() throws IOException { - delegate.close(); - } -} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java new file mode 100644 index 000000000000..78be448ec126 --- /dev/null +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java @@ -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.sort.io; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +import org.apache.spark.annotation.Private; +import org.apache.spark.shuffle.api.WritableByteChannelWrapper; + +/** + * :: Private :: + * Default implementation of {@link WritableByteChannelWrapper} that simply wraps a + * {@link WritableByteChannel} and closes the channel in {@link #close()}. + */ +@Private +public class DefaultWritableByteChannelWrapper implements WritableByteChannelWrapper { + + private final WritableByteChannel channel; + + public DefaultWritableByteChannelWrapper(WritableByteChannel channel) { + this.channel = channel; + } + + @Override + public WritableByteChannel channel() { + return channel; + } + + @Override + public void close() throws IOException { + channel.close(); + } +} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index 68d43d0036cb..c468b36698d6 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -17,10 +17,13 @@ package org.apache.spark.shuffle.sort.io; +import com.google.common.annotations.VisibleForTesting; + import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.api.ShuffleExecutorComponents; -import org.apache.spark.shuffle.api.ShuffleWriteSupport; +import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.storage.BlockManager; @@ -34,6 +37,16 @@ public LocalDiskShuffleExecutorComponents(SparkConf sparkConf) { this.sparkConf = sparkConf; } + @VisibleForTesting + public LocalDiskShuffleExecutorComponents( + SparkConf sparkConf, + BlockManager blockManager, + IndexShuffleBlockResolver blockResolver) { + this.sparkConf = sparkConf; + this.blockManager = blockManager; + this.blockResolver = blockResolver; + } + @Override public void initializeExecutor(String appId, String execId) { blockManager = SparkEnv.get().blockManager(); @@ -44,11 +57,17 @@ public void initializeExecutor(String appId, String execId) { } @Override - public ShuffleWriteSupport writes() { + public ShuffleMapOutputWriter createMapOutputWriter( + int shuffleId, + int mapId, + long mapTaskAttemptId, + int numPartitions, + ShuffleWriteMetricsReporter mapTaskWriteMetrics) { if (blockResolver == null) { throw new IllegalStateException( - "Executor components must be initialized before getting writers."); + "Executor components must be initialized before getting writers."); } - return new LocalDiskShuffleWriteSupport(sparkConf, blockResolver); + return new LocalDiskShuffleMapOutputWriter( + shuffleId, mapId, numPartitions, mapTaskWriteMetrics, blockResolver, sparkConf); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 6108fe36ab7f..4b9d92cd0dcb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,9 +31,8 @@ import org.apache.spark.SparkConf; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.api.ShufflePartitionWriter; -import org.apache.spark.shuffle.api.TransferrableWritableByteChannel; +import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.internal.config.package$; -import org.apache.spark.shuffle.sort.DefaultTransferrableWritableByteChannel; import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.storage.TimeTrackingOutputStream; @@ -40,8 +40,8 @@ /** * Implementation of {@link ShuffleMapOutputWriter} that replicates the functionality of shuffle - * persisting shuffle data to local disk alongside index files, identical to Spark's shuffle - * storage mechanism from Spark 2.4 and earlier. + * persisting shuffle data to local disk alongside index files, identical to Spark's historic + * canonical shuffle storage mechanism. */ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { @@ -171,7 +171,7 @@ public OutputStream openStream() throws IOException { } @Override - public TransferrableWritableByteChannel openTransferrableChannel() throws IOException { + public WritableByteChannelWrapper openChannelWrapper() throws IOException { if (partChannel == null) { if (partStream != null) { throw new IllegalStateException("Requested an output stream for a previous write but" + @@ -241,12 +241,11 @@ private void verifyNotClosed() { } } - private class PartitionWriterChannel extends DefaultTransferrableWritableByteChannel { + private class PartitionWriterChannel implements WritableByteChannelWrapper { private final int partitionId; PartitionWriterChannel(int partitionId) { - super(outputFileChannel); this.partitionId = partitionId; } @@ -255,6 +254,11 @@ public long getCount() throws IOException { return writtenPosition - currChannelPosition; } + @Override + public WritableByteChannel channel() { + return outputFileChannel; + } + @Override public void close() throws IOException { partitionLengths[partitionId] = getCount(); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java deleted file mode 100644 index ddb697338cc8..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleWriteSupport.java +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.sort.io; - -import org.apache.spark.SparkConf; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; -import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; -import org.apache.spark.shuffle.api.ShuffleWriteSupport; -import org.apache.spark.shuffle.IndexShuffleBlockResolver; - -public class LocalDiskShuffleWriteSupport implements ShuffleWriteSupport { - - private final SparkConf sparkConf; - private final IndexShuffleBlockResolver blockResolver; - - public LocalDiskShuffleWriteSupport( - SparkConf sparkConf, - IndexShuffleBlockResolver blockResolver) { - this.sparkConf = sparkConf; - this.blockResolver = blockResolver; - } - - @Override - public ShuffleMapOutputWriter createMapOutputWriter( - int shuffleId, - int mapId, - long mapTaskAttemptId, - int numPartitions, - ShuffleWriteMetricsReporter writeMetrics) { - return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, writeMetrics, blockResolver, sparkConf); - } -} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 2291e6b90cd8..cda3b57104d5 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -813,7 +813,7 @@ package object config { .createWithDefault(false) private[spark] val SHUFFLE_IO_PLUGIN_CLASS = - ConfigBuilder("spark.shuffle.io.plugin.class") + ConfigBuilder("spark.shuffle.sort.io.plugin.class") .doc("Name of the class to use for shuffle IO.") .stringConf .createWithDefault(classOf[LocalDiskShuffleDataIO].getName) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 763f090e420a..17719f516a0a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -155,7 +155,7 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager context.taskAttemptId(), env.conf, metrics, - shuffleExecutorComponents.writes()) + shuffleExecutorComponents) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => new SortShuffleWriter(shuffleBlockResolver, other, mapId, context) } diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index d107bdde7d75..f1e14a16d4fb 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.shuffle.sort import java.io.File -import java.util.{Properties, UUID} +import java.util.UUID import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.util.Random import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS @@ -36,8 +35,8 @@ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} import org.apache.spark.serializer.{JavaSerializer, SerializerInstance, SerializerManager} import org.apache.spark.shuffle.IndexShuffleBlockResolver -import org.apache.spark.shuffle.api.ShuffleWriteSupport -import org.apache.spark.shuffle.sort.io.LocalDiskShuffleWriteSupport +import org.apache.spark.shuffle.api.ShuffleExecutorComponents +import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage._ import org.apache.spark.util.Utils @@ -52,7 +51,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte private var taskMetrics: TaskMetrics = _ private var tempDir: File = _ private var outputFile: File = _ - private var writeSupport: ShuffleWriteSupport = _ + private var shuffleExecutorComponents: ShuffleExecutorComponents = _ private val conf: SparkConf = new SparkConf(loadDefaults = false) .set("spark.app.id", "sampleApp") private val temporaryFilesCreated: mutable.Buffer[File] = new ArrayBuffer[File]() @@ -121,8 +120,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte val memoryManager = new TestMemoryManager(conf) val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) - - writeSupport = new LocalDiskShuffleWriteSupport(conf, blockResolver) + shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( + conf, blockManager, blockResolver) } override def afterEach(): Unit = { @@ -144,7 +143,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte 0L, // MapTaskAttemptId conf, taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport) + shuffleExecutorComponents) writer.write(Iterator.empty) writer.stop( /* success = */ true) @@ -171,8 +170,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte 0L, transferConf, taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) + shuffleExecutorComponents) writer.write(records) writer.stop( /* success = */ true) assert(temporaryFilesCreated.nonEmpty) @@ -208,8 +206,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) + shuffleExecutorComponents) intercept[SparkException] { writer.write(records) @@ -231,8 +228,7 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte 0L, conf, taskContext.taskMetrics().shuffleWriteMetrics, - writeSupport - ) + shuffleExecutorComponents) intercept[SparkException] { writer.write((0 until 100000).iterator.map(i => { if (i == 99990) { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index 334b8be4fe7b..ed9ff10cac14 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.shuffle.sort.io import java.io.{File, FileInputStream, FileOutputStream} +import java.nio.channels.FileChannel import java.nio.file.Files import java.util.Arrays import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.ArgumentMatchers.{any, anyInt, anyLong} +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mock -import org.mockito.Mockito.{doAnswer, doNothing, when} +import org.mockito.Mockito.when import org.mockito.MockitoAnnotations import org.scalatest.BeforeAndAfterEach @@ -42,13 +43,11 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA private var shuffleWriteMetrics: ShuffleWriteMetrics = _ private val NUM_PARTITIONS = 4 - private val data: Array[Array[Byte]] = (0 until NUM_PARTITIONS).map { - p => { - if (p == 3) { - Array.emptyByteArray - } else { - (0 to p * 10).map(_ + p).map(_.toByte).toArray - } + private val data: Array[Array[Byte]] = (0 until NUM_PARTITIONS).map { p => + if (p == 3) { + Array.emptyByteArray + } else { + (0 to p * 10).map(_ + p).map(_.toByte).toArray } }.toArray @@ -71,7 +70,7 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA override def beforeEach(): Unit = { MockitoAnnotations.initMocks(this) - tempDir = Utils.createTempDir(null, "test") + tempDir = Utils.createTempDir() mergedOutputFile = File.createTempFile("mergedoutput", "", tempDir) tempFile = File.createTempFile("tempfile", "", tempDir) partitionSizesInMergedFile = null @@ -79,19 +78,17 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA .set("spark.app.id", "example.spark.app") .set("spark.shuffle.unsafe.file.output.buffer", "16k") when(blockResolver.getDataFile(anyInt, anyInt)).thenReturn(mergedOutputFile) - - doNothing().when(shuffleWriteMetrics).incWriteTime(anyLong) - - doAnswer { invocationOnMock => - partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - mergedOutputFile.delete - tmp.renameTo(mergedOutputFile) + when(blockResolver.writeIndexFileAndCommit( + anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + .thenAnswer { invocationOnMock => + partitionSizesInMergedFile = invocationOnMock.getArguments()(2).asInstanceOf[Array[Long]] + val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + mergedOutputFile.delete() + tmp.renameTo(mergedOutputFile) + } + null } - null - }.when(blockResolver) - .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) mapOutputWriter = new LocalDiskShuffleMapOutputWriter( 0, 0, @@ -123,13 +120,17 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA outputTempFileStream.write(data(p)) outputTempFileStream.close() val tempFileInput = new FileInputStream(outputTempFile) - val channel = writer.openTransferrableChannel() + val channel = writer.openChannelWrapper() Utils.tryWithResource(new FileInputStream(outputTempFile)) { tempFileInput => - Utils.tryWithResource(writer.openTransferrableChannel()) { channel => - channel.transferFrom(tempFileInput.getChannel, 0L, data(p).length) + Utils.tryWithResource(writer.openChannelWrapper()) { channelWrapper => + assert(channelWrapper.channel().isInstanceOf[FileChannel], + "Underlying channel should be a file channel") + Utils.copyFileStreamNIO( + tempFileInput.getChannel, channelWrapper.channel(), 0L, data(p).length) } } - assert(writer.getNumBytesWritten === data(p).length) + assert(writer.getNumBytesWritten === data(p).length, + s"Partition $p does not have the correct number of bytes.") } verifyWrittenRecords() } From e53a001b30a15ebc06df6b62c5650ae6f3213477 Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 24 Jul 2019 16:19:41 -0700 Subject: [PATCH 27/32] Code tags --- .../main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java index 4b5501f4dc63..e9e50ecc11e5 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleDataIO.java @@ -35,7 +35,7 @@ * consistent throughout most of Spark's history. *

* Alternative implementations of shuffle data storage can be loaded via setting - * spark.shuffle.io.plugin.class. + * spark.shuffle.sort.io.plugin.class. * @since 3.0.0 */ @Private From 56fa450b4d703c7bc34f338e2ed0cd21fc82a98c Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 24 Jul 2019 16:31:17 -0700 Subject: [PATCH 28/32] Add some docs --- .../apache/spark/shuffle/api/ShufflePartitionWriter.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index 8a94dfc1c7aa..cad7b8b654b1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -77,6 +77,12 @@ public interface ShufflePartitionWriter { *

* The default implementation should be sufficient for most situations. Only override this * method if there is a very specific optimization that needs to be built. + *

+ * 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 WritableByteChannelWrapper openChannelWrapper() throws IOException { return new DefaultWritableByteChannelWrapper(Channels.newChannel(openStream())); From b8b7b8d00418ccc735ec7bdcc10de5e71384e8bc Mon Sep 17 00:00:00 2001 From: mcheah Date: Wed, 24 Jul 2019 19:10:39 -0700 Subject: [PATCH 29/32] Change mockito format in BypassMergeSortShuffleWriterSuite --- .../BypassMergeSortShuffleWriterSuite.scala | 88 +++++++++---------- 1 file changed, 44 insertions(+), 44 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index f1e14a16d4fb..b9f81fa0d0a0 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -27,7 +27,6 @@ import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock import org.scalatest.BeforeAndAfterEach import org.apache.spark._ @@ -60,66 +59,67 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte override def beforeEach(): Unit = { super.beforeEach() + MockitoAnnotations.initMocks(this) tempDir = Utils.createTempDir() outputFile = File.createTempFile("shuffle", null, tempDir) taskMetrics = new TaskMetrics - MockitoAnnotations.initMocks(this) shuffleHandle = new BypassMergeSortShuffleHandle[Int, Int]( shuffleId = 0, numMaps = 2, dependency = dependency ) + val memoryManager = new TestMemoryManager(conf) + val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) when(dependency.partitioner).thenReturn(new HashPartitioner(7)) when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) - doAnswer { (invocationOnMock: InvocationOnMock) => - val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - outputFile.delete - tmp.renameTo(outputFile) + when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) + + when(blockResolver.writeIndexFileAndCommit( + anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File]))) + .thenAnswer { invocationOnMock => + val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) + } + null } - null - }.when(blockResolver) - .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) - when(blockManager.diskBlockManager).thenReturn(diskBlockManager) + when(blockManager.getDiskWriter( + any[BlockId], + any[File], + any[SerializerInstance], + anyInt(), + any[ShuffleWriteMetrics])) + .thenAnswer { invocation => + val args = invocation.getArguments + val manager = new SerializerManager(new JavaSerializer(conf), conf) + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + manager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[Int], + syncWrites = false, + args(4).asInstanceOf[ShuffleWriteMetrics], + blockId = args(0).asInstanceOf[BlockId]) + } + + when(diskBlockManager.createTempShuffleBlock()) + .thenAnswer { _ => + val blockId = new TempShuffleBlockId(UUID.randomUUID) + val file = new File(tempDir, blockId.name) + blockIdToFileMap.put(blockId, file) + temporaryFilesCreated += file + (blockId, file) + } - doAnswer((invocation: InvocationOnMock) => { - val args = invocation.getArguments - val manager = new SerializerManager(new JavaSerializer(conf), conf) - new DiskBlockObjectWriter( - args(1).asInstanceOf[File], - manager, - args(2).asInstanceOf[SerializerInstance], - args(3).asInstanceOf[Int], - syncWrites = false, - args(4).asInstanceOf[ShuffleWriteMetrics], - blockId = args(0).asInstanceOf[BlockId] - ) - }).when(blockManager) - .getDiskWriter( - any[BlockId], - any[File], - any[SerializerInstance], - anyInt(), - any[ShuffleWriteMetrics]) - - doAnswer((_: InvocationOnMock) => { - val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = new File(tempDir, blockId.name) - blockIdToFileMap.put(blockId, file) - temporaryFilesCreated += file - (blockId, file) - }).when(diskBlockManager).createTempShuffleBlock() - - doAnswer((invocation: InvocationOnMock) => { + when(diskBlockManager.getFile(any[BlockId])).thenAnswer { invocation => blockIdToFileMap(invocation.getArguments.head.asInstanceOf[BlockId]) - }).when(diskBlockManager).getFile(any[BlockId]) + } - val memoryManager = new TestMemoryManager(conf) - val taskMemoryManager = new TaskMemoryManager(memoryManager, 0) - when(taskContext.taskMemoryManager()).thenReturn(taskMemoryManager) shuffleExecutorComponents = new LocalDiskShuffleExecutorComponents( conf, blockManager, blockResolver) } From 2d2940448783ca9f8fa9f6f22b65dfc233c756aa Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 29 Jul 2019 13:18:39 -0700 Subject: [PATCH 30/32] Remove metrics from the API. --- .../api/ShuffleExecutorComponents.java | 18 ++---- .../shuffle/api/ShuffleMapOutputWriter.java | 4 +- .../shuffle/api/ShufflePartitionWriter.java | 4 +- .../shuffle/api/ShuffleWriteSupport.java | 57 ------------------- .../sort/BypassMergeSortShuffleWriter.java | 6 +- .../LocalDiskShuffleExecutorComponents.java | 6 +- .../io/LocalDiskShuffleMapOutputWriter.java | 9 +-- 7 files changed, 15 insertions(+), 89 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index 12be3b2dd0a9..a0720fd3b647 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -19,7 +19,6 @@ import java.io.IOException; import org.apache.spark.annotation.Private; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * :: Private :: @@ -39,24 +38,17 @@ public interface ShuffleExecutorComponents { /** * 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 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. + * 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. - * @param mapTaskWriteMetrics The map task's write metrics, which can be updated by the returned - * writer. The updates that are posted to this reporter are listed in - * the Spark UI. Note that the caller will update the total write time - * at the end of the map task, so implementations should not call - * {@link ShuffleWriteMetricsReporter#incWriteTime(long)}. +* these partitions may be empty. */ ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, long mapTaskAttemptId, - int numPartitions, - ShuffleWriteMetricsReporter mapTaskWriteMetrics) throws IOException; + int numPartitions) throws IOException; } diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java index 2341bb4d2939..45a593c25a93 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleMapOutputWriter.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.spark.annotation.Private; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * :: Private :: @@ -40,8 +39,7 @@ public interface ShuffleMapOutputWriter { * 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 ShuffleWriteSupport#createMapOutputWriter( - * int, int, long, int, ShuffleWriteMetricsReporter)}. + * {@link ShuffleExecutorComponents#createMapOutputWriter(int, int, long, int)}. *

* 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 diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index cad7b8b654b1..f9700ed00a67 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -79,8 +79,8 @@ public interface ShufflePartitionWriter { * method if there is a very specific optimization that needs to be built. *

* 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()}, + * 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)}. */ diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java deleted file mode 100644 index cdf06ce715ca..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleWriteSupport.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; - -/** - * :: Private :: - * A module that returns shuffle writers to persist data that is written by shuffle map tasks. - * - * @since 3.0.0 - */ -@Private -public interface ShuffleWriteSupport { - - /** - * 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. - * @param mapTaskWriteMetrics The map task's write metrics, which can be updated by the returned - * writer. The updates that are posted to this reporter are listed in - * the Spark UI. Note that the caller will update the total write time - * at the end of the map task, so implementations should not call - * {@link ShuffleWriteMetricsReporter#incWriteTime(long)}. - */ - ShuffleMapOutputWriter createMapOutputWriter( - int shuffleId, - int mapId, - long mapTaskAttemptId, - int numPartitions, - ShuffleWriteMetricsReporter mapTaskWriteMetrics) throws IOException; -} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index eed62585df4d..4420655f9ac1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -129,7 +129,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { public void write(Iterator> records) throws IOException { assert (partitionWriters == null); ShuffleMapOutputWriter mapOutputWriter = shuffleExecutorComponents - .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions, writeMetrics); + .createMapOutputWriter(shuffleId, mapId, mapTaskAttemptId, numPartitions); try { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; @@ -241,7 +241,9 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro logger.error("Unable to delete file for partition {}", i); } } - lengths[i] = writer.getNumBytesWritten(); + long numBytesWritten = writer.getNumBytesWritten(); + lengths[i] = numBytesWritten; + writeMetrics.incBytesWritten(numBytesWritten); } } finally { writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java index c468b36698d6..02eb71073728 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleExecutorComponents.java @@ -21,7 +21,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.SparkEnv; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.api.ShuffleExecutorComponents; import org.apache.spark.shuffle.api.ShuffleMapOutputWriter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; @@ -61,13 +60,12 @@ public ShuffleMapOutputWriter createMapOutputWriter( int shuffleId, int mapId, long mapTaskAttemptId, - int numPartitions, - ShuffleWriteMetricsReporter mapTaskWriteMetrics) { + int numPartitions) { if (blockResolver == null) { throw new IllegalStateException( "Executor components must be initialized before getting writers."); } return new LocalDiskShuffleMapOutputWriter( - shuffleId, mapId, numPartitions, mapTaskWriteMetrics, blockResolver, sparkConf); + shuffleId, mapId, numPartitions, blockResolver, sparkConf); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index 4b9d92cd0dcb..cf95e668adce 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -33,9 +33,7 @@ import org.apache.spark.shuffle.api.ShufflePartitionWriter; import org.apache.spark.shuffle.api.WritableByteChannelWrapper; import org.apache.spark.internal.config.package$; -import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; -import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.util.Utils; /** @@ -50,7 +48,6 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { private final int shuffleId; private final int mapId; - private final ShuffleWriteMetricsReporter metrics; private final IndexShuffleBlockResolver blockResolver; private final long[] partitionLengths; private final int bufferSize; @@ -61,19 +58,16 @@ public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { private File outputTempFile; private FileOutputStream outputFileStream; private FileChannel outputFileChannel; - private TimeTrackingOutputStream ts; private BufferedOutputStream outputBufferedFileStream; public LocalDiskShuffleMapOutputWriter( int shuffleId, int mapId, int numPartitions, - ShuffleWriteMetricsReporter metrics, IndexShuffleBlockResolver blockResolver, SparkConf sparkConf) { this.shuffleId = shuffleId; this.mapId = mapId; - this.metrics = metrics; this.blockResolver = blockResolver; this.bufferSize = (int) (long) sparkConf.get( @@ -130,10 +124,9 @@ private void cleanUp() throws IOException { private void initStream() throws IOException { if (outputFileStream == null) { outputFileStream = new FileOutputStream(outputTempFile, true); - ts = new TimeTrackingOutputStream(metrics, outputFileStream); } if (outputBufferedFileStream == null) { - outputBufferedFileStream = new BufferedOutputStream(ts, bufferSize); + outputBufferedFileStream = new BufferedOutputStream(outputFileStream, bufferSize); } } From 06ea01afc0660025564acff8f1d1e695a72a35cc Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 29 Jul 2019 15:49:42 -0700 Subject: [PATCH 31/32] Address more comments. --- .../api/ShuffleExecutorComponents.java | 1 + .../shuffle/api/ShufflePartitionWriter.java | 16 ++--- .../sort/BypassMergeSortShuffleWriter.java | 72 +++++++++++-------- .../io/DefaultWritableByteChannelWrapper.java | 49 ------------- .../io/LocalDiskShuffleMapOutputWriter.java | 5 +- ...LocalDiskShuffleMapOutputWriterSuite.scala | 13 +--- 6 files changed, 57 insertions(+), 99 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java index a0720fd3b647..70c112b78911 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShuffleExecutorComponents.java @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.api; import java.io.IOException; + import org.apache.spark.annotation.Private; /** diff --git a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java index f9700ed00a67..928875156a70 100644 --- a/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/api/ShufflePartitionWriter.java @@ -18,11 +18,10 @@ package org.apache.spark.shuffle.api; import java.io.IOException; +import java.util.Optional; import java.io.OutputStream; -import java.nio.channels.Channels; import org.apache.spark.annotation.Private; -import org.apache.spark.shuffle.sort.io.DefaultWritableByteChannelWrapper; /** * :: Private :: @@ -67,16 +66,15 @@ public interface ShufflePartitionWriter { * 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 it + * {@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)}. *

* 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. - *

- * The default implementation should be sufficient for most situations. Only override this - * method if there is a very specific optimization that needs to be built. + * 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()}. *

* Note that the returned {@link WritableByteChannelWrapper} itself is closed, but not the * underlying channel that is returned by {@link WritableByteChannelWrapper#channel()}. Ensure @@ -84,8 +82,8 @@ public interface ShufflePartitionWriter { * {@link ShuffleMapOutputWriter#commitAllPartitions()}, or * {@link ShuffleMapOutputWriter#abort(Throwable)}. */ - default WritableByteChannelWrapper openChannelWrapper() throws IOException { - return new DefaultWritableByteChannelWrapper(Channels.newChannel(openStream())); + default Optional openChannelWrapper() throws IOException { + return Optional.empty(); } /** diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 4420655f9ac1..19e9d91dd6c4 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.channels.FileChannel; +import java.util.Optional; import javax.annotation.Nullable; import scala.None$; @@ -205,45 +206,23 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro final File file = partitionWriterSegments[i].file(); ShufflePartitionWriter writer = mapOutputWriter.getPartitionWriter(i); if (file.exists()) { - boolean copyThrewException = true; if (transferToEnabled) { - FileInputStream in = new FileInputStream(file); // Using WritableByteChannelWrapper to make resource closing consistent between // this implementation and UnsafeShuffleWriter. - try { - WritableByteChannelWrapper outputChannel = writer.openChannelWrapper(); - try (FileChannel inputChannel = in.getChannel()) { - Utils.copyFileStreamNIO( - inputChannel, outputChannel.channel(), 0L, inputChannel.size()); - copyThrewException = false; - } finally { - Closeables.close(outputChannel, copyThrewException); - } - } finally { - Closeables.close(in, copyThrewException); + Optional maybeOutputChannel = writer.openChannelWrapper(); + if (maybeOutputChannel.isPresent()) { + writePartitionedDataWithChannel(file, maybeOutputChannel.get()); + } else { + writePartitionedDataWithStream(file, writer); } } else { - FileInputStream in = new FileInputStream(file); - OutputStream outputStream; - try { - outputStream = writer.openStream(); - try { - Utils.copyStream(in, outputStream, false, false); - copyThrewException = false; - } finally { - Closeables.close(outputStream, copyThrewException); - } - } finally { - Closeables.close(in, copyThrewException); - } + writePartitionedDataWithStream(file, writer); } if (!file.delete()) { logger.error("Unable to delete file for partition {}", i); } } - long numBytesWritten = writer.getNumBytesWritten(); - lengths[i] = numBytesWritten; - writeMetrics.incBytesWritten(numBytesWritten); + lengths[i] = writer.getNumBytesWritten(); } } finally { writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); @@ -252,6 +231,41 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro return lengths; } + private void writePartitionedDataWithChannel( + File file, WritableByteChannelWrapper outputChannel) throws IOException { + boolean copyThrewException = true; + try { + FileInputStream in = new FileInputStream(file); + try (FileChannel inputChannel = in.getChannel()) { + Utils.copyFileStreamNIO( + inputChannel, outputChannel.channel(), 0L, inputChannel.size()); + copyThrewException = false; + } finally { + Closeables.close(in, copyThrewException); + } + } finally { + Closeables.close(outputChannel, copyThrewException); + } + } + + private void writePartitionedDataWithStream(File file, ShufflePartitionWriter writer) + throws IOException { + boolean copyThrewException = true; + FileInputStream in = new FileInputStream(file); + OutputStream outputStream; + try { + outputStream = writer.openStream(); + try { + Utils.copyStream(in, outputStream, false, false); + copyThrewException = false; + } finally { + Closeables.close(outputStream, copyThrewException); + } + } finally { + Closeables.close(in, copyThrewException); + } + } + @Override public Option stop(boolean success) { if (stopping) { diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java deleted file mode 100644 index 78be448ec126..000000000000 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/DefaultWritableByteChannelWrapper.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.sort.io; - -import java.io.IOException; -import java.nio.channels.WritableByteChannel; - -import org.apache.spark.annotation.Private; -import org.apache.spark.shuffle.api.WritableByteChannelWrapper; - -/** - * :: Private :: - * Default implementation of {@link WritableByteChannelWrapper} that simply wraps a - * {@link WritableByteChannel} and closes the channel in {@link #close()}. - */ -@Private -public class DefaultWritableByteChannelWrapper implements WritableByteChannelWrapper { - - private final WritableByteChannel channel; - - public DefaultWritableByteChannelWrapper(WritableByteChannel channel) { - this.channel = channel; - } - - @Override - public WritableByteChannel channel() { - return channel; - } - - @Override - public void close() throws IOException { - channel.close(); - } -} diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java index cf95e668adce..add4634a61fb 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java @@ -25,6 +25,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; +import java.util.Optional; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -164,7 +165,7 @@ public OutputStream openStream() throws IOException { } @Override - public WritableByteChannelWrapper openChannelWrapper() throws IOException { + public Optional openChannelWrapper() throws IOException { if (partChannel == null) { if (partStream != null) { throw new IllegalStateException("Requested an output stream for a previous write but" + @@ -174,7 +175,7 @@ public WritableByteChannelWrapper openChannelWrapper() throws IOException { initChannel(); partChannel = new PartitionWriterChannel(partitionId); } - return partChannel; + return Optional.of(partChannel); } @Override diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala index ed9ff10cac14..5693b9824523 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriterSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle.sort.io -import java.io.{File, FileInputStream, FileOutputStream} +import java.io.{File, FileInputStream} import java.nio.channels.FileChannel import java.nio.file.Files import java.util.Arrays @@ -30,7 +30,6 @@ import org.mockito.MockitoAnnotations import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.shuffle.IndexShuffleBlockResolver import org.apache.spark.util.Utils @@ -39,9 +38,6 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA @Mock(answer = RETURNS_SMART_NULLS) private var blockResolver: IndexShuffleBlockResolver = _ - @Mock(answer = RETURNS_SMART_NULLS) - private var shuffleWriteMetrics: ShuffleWriteMetrics = _ - private val NUM_PARTITIONS = 4 private val data: Array[Array[Byte]] = (0 until NUM_PARTITIONS).map { p => if (p == 3) { @@ -93,7 +89,6 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA 0, 0, NUM_PARTITIONS, - shuffleWriteMetrics, blockResolver, conf) } @@ -116,13 +111,11 @@ class LocalDiskShuffleMapOutputWriterSuite extends SparkFunSuite with BeforeAndA (0 until NUM_PARTITIONS).foreach { p => val writer = mapOutputWriter.getPartitionWriter(p) val outputTempFile = File.createTempFile("channelTemp", "", tempDir) - val outputTempFileStream = new FileOutputStream(outputTempFile) - outputTempFileStream.write(data(p)) - outputTempFileStream.close() + Files.write(outputTempFile.toPath, data(p)) val tempFileInput = new FileInputStream(outputTempFile) val channel = writer.openChannelWrapper() Utils.tryWithResource(new FileInputStream(outputTempFile)) { tempFileInput => - Utils.tryWithResource(writer.openChannelWrapper()) { channelWrapper => + Utils.tryWithResource(writer.openChannelWrapper().get) { channelWrapper => assert(channelWrapper.channel().isInstanceOf[FileChannel], "Underlying channel should be a file channel") Utils.copyFileStreamNIO( From 7dceec971784049442ec3d4cb71ddaa225e1e21f Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 30 Jul 2019 11:13:50 -0700 Subject: [PATCH 32/32] Args per line --- .../spark/shuffle/sort/BypassMergeSortShuffleWriter.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index 19e9d91dd6c4..3ccee703619b 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -232,7 +232,8 @@ private long[] writePartitionedData(ShuffleMapOutputWriter mapOutputWriter) thro } private void writePartitionedDataWithChannel( - File file, WritableByteChannelWrapper outputChannel) throws IOException { + File file, + WritableByteChannelWrapper outputChannel) throws IOException { boolean copyThrewException = true; try { FileInputStream in = new FileInputStream(file);