-
Notifications
You must be signed in to change notification settings - Fork 6
[SPARK-25299] Implement default version of the API for shuffle writes #6
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 1 commit
7160ce3
460f0ea
96d1774
64fb327
996e903
3b9d33c
1f1c159
0737515
1ded83d
3353155
7a79bd9
9e3f05c
9f6230b
14df750
8cf80f7
46a0174
1325903
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,26 @@ | ||
| /* | ||
| * 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.OutputStream; | ||
|
|
||
| public abstract class DefaultShuffleBlockOutputStream extends OutputStream { | ||
|
|
||
| public abstract int getCount(); | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.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); | ||
| } | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,57 @@ | ||
| /* | ||
| * 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.TaskContext; | ||
| import org.apache.spark.api.shuffle.ShuffleExecutorComponents; | ||
| import org.apache.spark.api.shuffle.ShuffleWriteSupport; | ||
| import org.apache.spark.executor.TaskMetrics; | ||
| 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; | ||
| private TaskMetrics metrics; | ||
|
|
||
| public DefaultShuffleExecutorComponents(SparkConf sparkConf) { | ||
| this.sparkConf = sparkConf; | ||
| } | ||
|
|
||
| @Override | ||
| public void intitializeExecutor(String appId, String execId) { | ||
| blockManager = SparkEnv.get().blockManager(); | ||
| blockResolver = new IndexShuffleBlockResolver(sparkConf, blockManager); | ||
| metrics = TaskContext.get().taskMetrics(); | ||
|
||
| } | ||
|
|
||
| @Override | ||
| public ShuffleWriteSupport writes() { | ||
| if (blockResolver == null || metrics == null) { | ||
| throw new IllegalStateException( | ||
| "Executor components must be initialized before getting writers."); | ||
| } | ||
|
|
||
| return new DefaultShuffleWriteSupport( | ||
| sparkConf, blockResolver, metrics.shuffleWriteMetrics()); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,171 @@ | ||
| /* | ||
| * 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 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 int numPartitions; | ||
| private final ShuffleWriteMetricsReporter metrics; | ||
| private final IndexShuffleBlockResolver blockResolver; | ||
| private final long[] partitionLengths; | ||
| private ShufflePartitionWriter[] partitionWriters; | ||
| private final int bufferSize; | ||
| private int currPartitionId = 0; | ||
|
|
||
| private final File outputFile; | ||
| private final File outputTempFile; | ||
| private FileOutputStream outputFileStream; | ||
| 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.numPartitions = numPartitions; | ||
| 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.partitionWriters = new ShufflePartitionWriter[numPartitions]; | ||
| this.outputFile = blockResolver.getDataFile(shuffleId, mapId); | ||
| this.outputTempFile = Utils.tempFileWith(outputFile); | ||
| } | ||
|
|
||
| @Override | ||
| public ShufflePartitionWriter getNextPartitionWriter() throws IOException { | ||
| initStream(); | ||
| ShufflePartitionWriter shufflePartitionWriter = | ||
| new DefaultShufflePartitionWriter( | ||
| new DefaultShuffleBlockOutputStreamImpl()); | ||
| partitionWriters[currPartitionId++] = shufflePartitionWriter; | ||
| return shufflePartitionWriter; | ||
| } | ||
|
|
||
| @Override | ||
| public void commitAllPartitions() throws IOException { | ||
ifilonenko marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| for (int pId = 0; pId < numPartitions; pId ++) { | ||
| partitionLengths[pId] = partitionWriters[pId].getLength(); | ||
| } | ||
| blockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, outputTempFile); | ||
| // TODO: Maybe monitor a little more intelligably then checking files? | ||
| if (!outputFile.getParentFile().isDirectory() && !outputFile.getParentFile().mkdirs()) { | ||
| throw new IOException( | ||
| String.format( | ||
| "Failed to create shuffle file directory at %s.", | ||
| outputFile.getParentFile().getAbsolutePath())); | ||
| } | ||
| if (!outputFile.isFile() && !outputFile.createNewFile()) { | ||
| throw new IOException( | ||
| String.format( | ||
| "Failed to create empty shuffle file at %s.", outputFile.getAbsolutePath())); | ||
| } | ||
ifilonenko marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public void abort(Throwable error) throws IOException { | ||
| cleanUp(); | ||
|
||
| if (!outputTempFile.delete()) { | ||
|
||
| log.warn("Failed to delete temporary shuffle file at {}", outputTempFile.getAbsolutePath()); | ||
| } | ||
| if (!outputFile.delete()) { | ||
| log.warn("Failed to delete outputshuffle file at {}", outputTempFile.getAbsolutePath()); | ||
| } | ||
| } | ||
|
|
||
| private void cleanUp() throws IOException { | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. You're closing the streams, but I noticed that There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Heh I am wrong... closing the fileOutputStream closes the channel :P |
||
| if (outputBufferedFileStream != null) { | ||
| outputBufferedFileStream.flush(); | ||
|
||
| outputBufferedFileStream.close(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Closing the buffered output stream will close the underlying file stream. Is closing the same file output stream twice prone to throwing an error? If it doesn't throw an error than this is fine (would rather be explicit about all the resources we're closing). |
||
| } | ||
|
|
||
| if (outputFileStream != null) { | ||
| outputFileStream.flush(); | ||
|
||
| outputFileStream.close(); | ||
|
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Does this close the file channel as well? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yea, looking at the implementation of |
||
| } | ||
| } | ||
|
|
||
| private void initStream() throws IOException { | ||
| if (outputFileStream == null) { | ||
| outputFileStream = new FileOutputStream(outputTempFile, true); | ||
| ts = new TimeTrackingOutputStream(metrics, outputFileStream); | ||
ifilonenko marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| } | ||
| if (outputBufferedFileStream == null) { | ||
| outputBufferedFileStream = new BufferedOutputStream(ts, bufferSize); | ||
|
||
| } | ||
| } | ||
|
|
||
| private class DefaultShuffleBlockOutputStreamImpl extends DefaultShuffleBlockOutputStream { | ||
|
|
||
| private boolean isClosed = false; | ||
| private int count = 0; | ||
|
|
||
| @Override | ||
| public int getCount() { | ||
| return count; | ||
| } | ||
|
|
||
| @Override | ||
| public void write(int b) throws IOException { | ||
| if (isClosed) { | ||
| throw new IllegalStateException("Attempting to write to a closed block byte channel."); | ||
| } | ||
| outputBufferedFileStream.write(b); | ||
| count++; | ||
| } | ||
|
|
||
| @Override | ||
| public void close() throws IOException { | ||
| flush(); | ||
| isClosed = true; | ||
| } | ||
|
|
||
| @Override | ||
| public void flush() throws IOException { | ||
| outputBufferedFileStream.flush(); | ||
| } | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,52 @@ | ||
| /* | ||
| * 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.api.shuffle.ShufflePartitionWriter; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.io.*; | ||
|
|
||
| public class DefaultShufflePartitionWriter implements ShufflePartitionWriter { | ||
|
||
|
|
||
| private static final Logger log = | ||
| LoggerFactory.getLogger(DefaultShufflePartitionWriter.class); | ||
|
|
||
| private final DefaultShuffleBlockOutputStream stream; | ||
|
|
||
| public DefaultShufflePartitionWriter( | ||
| DefaultShuffleBlockOutputStream stream) { | ||
| this.stream = stream; | ||
| } | ||
|
|
||
| @Override | ||
| public OutputStream openStream() throws IOException { | ||
| return stream; | ||
|
||
| } | ||
|
|
||
| @Override | ||
| public long getLength() { | ||
| try { | ||
| stream.close(); | ||
| } catch (Exception e) { | ||
| log.error("Error with closing stream for partition", e); | ||
| } | ||
| return stream.getCount(); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,50 @@ | ||
| /* | ||
| * 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.ShuffleMapOutputWriter; | ||
| import org.apache.spark.api.shuffle.ShuffleWriteSupport; | ||
| import org.apache.spark.shuffle.IndexShuffleBlockResolver; | ||
| import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; | ||
|
|
||
| public class DefaultShuffleWriteSupport implements ShuffleWriteSupport { | ||
|
|
||
| private final SparkConf sparkConf; | ||
| private final IndexShuffleBlockResolver blockResolver; | ||
| private final ShuffleWriteMetricsReporter metrics; | ||
|
|
||
| public DefaultShuffleWriteSupport( | ||
| SparkConf sparkConf, | ||
|
||
| IndexShuffleBlockResolver blockResolver, | ||
| ShuffleWriteMetricsReporter metrics) { | ||
| this.sparkConf = sparkConf; | ||
| this.blockResolver = blockResolver; | ||
| this.metrics = metrics; | ||
| } | ||
|
|
||
| @Override | ||
| public ShuffleMapOutputWriter createMapOutputWriter( | ||
| String appId, | ||
| int shuffleId, | ||
| int mapId, | ||
| int numPartitions) { | ||
| return new DefaultShuffleMapOutputWriter( | ||
| shuffleId, mapId, numPartitions, metrics, blockResolver, sparkConf); | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why can't it just be a class with the implementation contents instead of an abstract class? Will there ever need to be other implementations of this?