boy-uber commented on a change in pull request #25007: [SPARK-28209][CORE][SHUFFLE] Proposed new shuffle writer API URL: https://github.com/apache/spark/pull/25007#discussion_r308374489
########## File path: core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java ########## @@ -0,0 +1,267 @@ +/* + * 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 java.nio.channels.WritableByteChannel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.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; + +/** + * Implementation of {@link ShuffleMapOutputWriter} that replicates the functionality of shuffle + * persisting shuffle data to local disk alongside index files, identical to Spark's historic + * canonical shuffle storage mechanism. + */ +public class LocalDiskShuffleMapOutputWriter implements ShuffleMapOutputWriter { + + private static final Logger log = + LoggerFactory.getLogger(LocalDiskShuffleMapOutputWriter.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 lastPartitionId = -1; + private long currChannelPosition; + + private final File outputFile; + 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( + 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 getPartitionWriter(int reducePartitionId) throws IOException { + if (reducePartitionId <= lastPartitionId) { + throw new IllegalArgumentException("Partitions should be requested in increasing order."); + } + lastPartitionId = reducePartitionId; + if (outputTempFile == null) { + outputTempFile = Utils.tempFileWith(outputFile); + } + if (outputFileChannel != null) { + currChannelPosition = outputFileChannel.position(); + } else { + currChannelPosition = 0L; + } + return new LocalDiskShufflePartitionWriter(reducePartitionId); + } + + @Override + public void commitAllPartitions() throws IOException { + cleanUp(); + File resolvedTmp = outputTempFile != null && outputTempFile.isFile() ? outputTempFile : null; + blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, resolvedTmp); + } + + @Override + 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()); + } + } + + 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 LocalDiskShufflePartitionWriter implements ShufflePartitionWriter { + + private final int partitionId; + private PartitionWriterStream partStream = null; + private PartitionWriterChannel partChannel = null; + + private LocalDiskShufflePartitionWriter(int partitionId) { + this.partitionId = partitionId; + } + + @Override + 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); + } + return partStream; Review comment: Feel a little uncomfortable to return internal field "partStream" to outside of this class. Is it possible to modify the design here to avoid returning internal field? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
