This is an automated email from the ASF dual-hosted git repository.
roryqi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-uniffle.git
The following commit(s) were added to refs/heads/master by this push:
new 7b43bcea4 [#1246] feat(tez): Support remote spill for unordered input.
(#1250)
7b43bcea4 is described below
commit 7b43bcea4c34648ba51138c34e8d70f6b4482fd7
Author: zhengchenyu <[email protected]>
AuthorDate: Tue Oct 31 17:39:11 2023 +0800
[#1246] feat(tez): Support remote spill for unordered input. (#1250)
### What changes were proposed in this pull request?
When unordered input fetch data from shuffle server, if the fetched data is
too large, will spill data to local filesystem.
This PR provide another way: spill data to remote filesystem.
### Why are the changes needed?
Fix: #1246
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
unit test and tez example test on cluster.
---
.../library/common/shuffle/RemoteFetchedInput.java | 153 ++++++++++++++++++
.../impl/RssSimpleFetchedInputAllocator.java | 114 ++++++++++++--
.../shuffle/orderedgrouped/RssTezBypassWriter.java | 3 +-
.../runtime/library/input/RssUnorderedKVInput.java | 3 +-
.../impl/RssSimpleFetchedInputAllocatorTest.java | 56 +++----
.../common/shuffle/impl/RssTezFetcherTest.java | 175 +++++++++++++++++++++
.../orderedgrouped/RssTezBypassWriterTest.java | 108 ++++++++++++-
.../apache/uniffle/common/RemoteStorageInfo.java | 9 +-
8 files changed, 572 insertions(+), 49 deletions(-)
diff --git
a/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/RemoteFetchedInput.java
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/RemoteFetchedInput.java
new file mode 100644
index 000000000..2c02513d9
--- /dev/null
+++
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/RemoteFetchedInput.java
@@ -0,0 +1,153 @@
+/*
+ * 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.tez.runtime.library.common.shuffle;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RemoteFetchedInput extends FetchedInput {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(RemoteFetchedInput.class);
+
+ /*
+ * Output path must be unique, is composed of unique_id, src_id and spill_id.
+ * unique_id is task attempt id + io context increasing id.
+ * src_id the source input id, is determined by partition and source task.
+ * spill_id is the spill id of map task.
+ * */
+ private static final String OUTPUT_FILE_PATTERN =
"%s/%s_src_%d_spill_%d.out";
+
+ private final FileSystem remoteFS;
+ private final Path tmpOutputPath;
+ private final Path outputPath;
+ private final long size;
+
+ public RemoteFetchedInput(
+ long compressedSize,
+ InputAttemptIdentifier inputAttemptIdentifier,
+ FetchedInputCallback callbackHandler,
+ FileSystem remoteFS,
+ String remoteSpillBasePath,
+ String uniqueId,
+ String appAttemptId) {
+ super(inputAttemptIdentifier, callbackHandler);
+ this.size = compressedSize;
+ this.remoteFS = remoteFS;
+ String outputFile =
+ String.format(
+ OUTPUT_FILE_PATTERN,
+ appAttemptId,
+ uniqueId,
+ this.getInputAttemptIdentifier().getInputIdentifier(),
+ this.getInputAttemptIdentifier().getSpillEventId());
+ this.outputPath = new Path(remoteSpillBasePath, outputFile);
+ // Files are not clobbered due to the id being appended to the outputPath
in the tmpPath,
+ // otherwise fetches for the same task but from different attempts would
clobber each other.
+ this.tmpOutputPath = outputPath.suffix(String.valueOf(getId()));
+ }
+
+ @Override
+ public Type getType() {
+ return Type.DISK;
+ }
+
+ @Override
+ public long getSize() {
+ return size;
+ }
+
+ @Override
+ public OutputStream getOutputStream() throws IOException {
+ return remoteFS.create(tmpOutputPath);
+ }
+
+ @Override
+ public InputStream getInputStream() throws IOException {
+ return remoteFS.open(outputPath);
+ }
+
+ public final Path getInputPath() {
+ if (isState(State.COMMITTED)) {
+ return this.outputPath;
+ }
+ return this.tmpOutputPath;
+ }
+
+ @Override
+ public void commit() throws IOException {
+ if (isState(State.PENDING)) {
+ setState(State.COMMITTED);
+ remoteFS.rename(tmpOutputPath, outputPath);
+ notifyFetchComplete();
+ }
+ }
+
+ @Override
+ public void abort() throws IOException {
+ if (isState(State.PENDING)) {
+ setState(State.ABORTED);
+ // TODO NEWTEZ Maybe defer this to container cleanup
+ remoteFS.delete(tmpOutputPath, false);
+ notifyFetchFailure();
+ }
+ }
+
+ @Override
+ public void free() {
+ Preconditions.checkState(
+ isState(State.COMMITTED) || isState(State.ABORTED),
+ "FetchedInput can only be freed after it is committed or aborted");
+ if (isState(State.COMMITTED)) {
+ setState(State.FREED);
+ try {
+ // TODO NEWTEZ Maybe defer this to container cleanup
+ remoteFS.delete(outputPath, false);
+ } catch (IOException e) {
+ // Ignoring the exception, will eventually be cleaned by container
+ // cleanup.
+ LOG.warn("Failed to remvoe file : " + outputPath.toString());
+ }
+ notifyFreedResource();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "RemoteFetchedInput [outputPath="
+ + outputPath
+ + ", inputAttemptIdentifier="
+ + getInputAttemptIdentifier()
+ + ", actualSize="
+ + getSize()
+ + ", type="
+ + getType()
+ + ", id="
+ + getId()
+ + ", state="
+ + getState()
+ + "]";
+ }
+}
diff --git
a/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocator.java
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocator.java
index 0e7ef92c6..584c5718b 100644
---
a/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocator.java
+++
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocator.java
@@ -18,10 +18,16 @@
package org.apache.tez.runtime.library.common.shuffle.impl;
import java.io.IOException;
+import java.util.Map;
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.common.RssTezConfig;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.common.Constants;
@@ -30,17 +36,25 @@ import
org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput;
import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
import org.apache.tez.runtime.library.common.shuffle.FetchedInput.Type;
import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput;
+import org.apache.tez.runtime.library.common.shuffle.RemoteFetchedInput;
import
org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.uniffle.common.RemoteStorageInfo;
import org.apache.uniffle.common.exception.RssException;
+import org.apache.uniffle.common.filesystem.HadoopFilesystemProvider;
+
+import static org.apache.tez.common.RssTezConfig.RSS_REMOTE_SPILL_STORAGE_PATH;
/** Usage: Create instance, setInitialMemoryAvailable(long),
configureAndStart() */
@Private
public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator {
private static final Logger LOG =
LoggerFactory.getLogger(RssSimpleFetchedInputAllocator.class);
+ // In order to be compatible with the Tez IFile file format, the decoded
data needs to be added
+ // with the corresponding HEADER and CHECKSUM, which occupy 8 bytes.
+ private static final int IFILE_HEAD_CHECKSUM_LEN = 8;
private final Configuration conf;
@@ -58,18 +72,27 @@ public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator
private volatile long usedMemory = 0;
+ private final String uniqueIdentifier;
+ private final String appAttemptId;
+ private final boolean remoteSpillEnable;
+ private FileSystem remoteFS;
+ private String remoteSpillBasePath;
+
public RssSimpleFetchedInputAllocator(
String srcNameTrimmed,
String uniqueIdentifier,
int dagID,
Configuration conf,
long maxTaskAvailableMemory,
- long memoryAvailable) {
+ long memoryAvailable,
+ String appAttemptId) {
super(srcNameTrimmed, uniqueIdentifier, dagID, conf,
maxTaskAvailableMemory, memoryAvailable);
this.srcNameTrimmed = srcNameTrimmed;
this.conf = conf;
this.maxAvailableTaskMemory = maxTaskAvailableMemory;
this.initialMemoryAvailable = memoryAvailable;
+ this.uniqueIdentifier = uniqueIdentifier;
+ this.appAttemptId = appAttemptId;
this.fileNameAllocator = new TezTaskOutputFiles(conf, uniqueIdentifier,
dagID);
this.localDirAllocator = new
LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS);
@@ -113,6 +136,38 @@ public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator
}
this.maxSingleShuffleLimit =
(long) Math.min((memoryLimit * singleShuffleMemoryLimitPercent),
Integer.MAX_VALUE);
+ this.remoteSpillEnable =
conf.getBoolean(RssTezConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED, false);
+ if (this.remoteSpillEnable) {
+ this.remoteSpillBasePath = conf.get(RSS_REMOTE_SPILL_STORAGE_PATH);
+ if (StringUtils.isBlank(this.remoteSpillBasePath)) {
+ throw new RssException("You must set remote spill path!");
+ }
+ // construct remote configuration
+ String remoteStorageConf =
this.conf.get(RssTezConfig.RSS_REMOTE_STORAGE_CONF);
+ Map<String, String> remoteStorageConfMap =
+ RemoteStorageInfo.parseRemoteStorageConf(remoteStorageConf);
+ Configuration remoteConf = new Configuration(this.conf);
+ for (Map.Entry<String, String> entry : remoteStorageConfMap.entrySet()) {
+ remoteConf.set(entry.getKey(), entry.getValue());
+ }
+ // construct remote filesystem
+ int replication =
+ this.conf.getInt(
+ RssTezConfig.RSS_REDUCE_REMOTE_SPILL_REPLICATION,
+ RssTezConfig.RSS_REDUCE_REMOTE_SPILL_REPLICATION_DEFAULT);
+ int retries =
+ this.conf.getInt(
+ RssTezConfig.RSS_REDUCE_REMOTE_SPILL_RETRIES,
+ RssTezConfig.RSS_REDUCE_REMOTE_SPILL_RETRIES_DEFAULT);
+ try {
+ remoteConf.setInt("dfs.replication", replication);
+ remoteConf.setInt("dfs.client.block.write.retries", retries);
+ remoteFS =
+ HadoopFilesystemProvider.getFilesystem(new
Path(this.remoteSpillBasePath), remoteConf);
+ } catch (Exception e) {
+ throw new RssException("Cannot init remoteFS on path:" +
this.remoteSpillBasePath);
+ }
+ }
LOG.info(
srcNameTrimmed
@@ -150,9 +205,26 @@ public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator
long actualSize, long compressedSize, InputAttemptIdentifier
inputAttemptIdentifier)
throws IOException {
if (actualSize > maxSingleShuffleLimit || this.usedMemory + actualSize >
this.memoryLimit) {
- LOG.info("Allocate DiskFetchedInput, length:{}", actualSize);
- return new DiskFetchedInput(
- actualSize + 8, inputAttemptIdentifier, this, conf,
localDirAllocator, fileNameAllocator);
+ if (remoteSpillEnable) {
+ LOG.info("Allocate RemoteFetchedInput, length:{}", actualSize);
+ return new RemoteFetchedInput(
+ actualSize + IFILE_HEAD_CHECKSUM_LEN,
+ inputAttemptIdentifier,
+ this,
+ remoteFS,
+ remoteSpillBasePath,
+ uniqueIdentifier,
+ appAttemptId);
+ } else {
+ LOG.info("Allocate DiskFetchedInput, length:{}", actualSize);
+ return new DiskFetchedInput(
+ actualSize + IFILE_HEAD_CHECKSUM_LEN,
+ inputAttemptIdentifier,
+ this,
+ conf,
+ localDirAllocator,
+ fileNameAllocator);
+ }
} else {
this.usedMemory += actualSize;
if (LOG.isDebugEnabled()) {
@@ -178,14 +250,27 @@ public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator
switch (type) {
case DISK:
- LOG.info("AllocateType DiskFetchedInput, compressedSize:{}",
compressedSize);
- return new DiskFetchedInput(
- compressedSize + 8,
- inputAttemptIdentifier,
- this,
- conf,
- localDirAllocator,
- fileNameAllocator);
+ // It should not be called here.
+ if (remoteSpillEnable) {
+ LOG.info("AllocateType RemoteFetchedInput, compressedSize:{}",
compressedSize);
+ return new RemoteFetchedInput(
+ actualSize + IFILE_HEAD_CHECKSUM_LEN,
+ inputAttemptIdentifier,
+ this,
+ remoteFS,
+ remoteSpillBasePath,
+ uniqueIdentifier,
+ appAttemptId);
+ } else {
+ LOG.info("AllocateType DiskFetchedInput, compressedSize:{}",
compressedSize);
+ return new DiskFetchedInput(
+ actualSize + IFILE_HEAD_CHECKSUM_LEN,
+ inputAttemptIdentifier,
+ this,
+ conf,
+ localDirAllocator,
+ fileNameAllocator);
+ }
default:
return allocate(actualSize, compressedSize, inputAttemptIdentifier);
}
@@ -234,4 +319,9 @@ public class RssSimpleFetchedInputAllocator extends
SimpleFetchedInputAllocator
LOG.debug(srcNameTrimmed + ": " + "Used memory after freeing " + size +
" : " + usedMemory);
}
}
+
+ @VisibleForTesting
+ public void setRemoteFS(FileSystem remoteFS) {
+ this.remoteFS = remoteFS;
+ }
}
diff --git
a/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriter.java
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriter.java
index 4354c76ab..83419a348 100644
---
a/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriter.java
+++
b/client-tez/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriter.java
@@ -21,7 +21,6 @@ import java.io.IOException;
import java.io.OutputStream;
import com.google.common.primitives.Ints;
-import org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput;
import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput;
import org.slf4j.Logger;
@@ -73,7 +72,7 @@ public class RssTezBypassWriter {
byte[] memory = ((MemoryFetchedInput) mapOutput).getBytes();
System.arraycopy(buffer, 0, memory, 0, buffer.length);
} else if (mapOutput.getType() == FetchedInput.Type.DISK) {
- OutputStream output = ((DiskFetchedInput) mapOutput).getOutputStream();
+ OutputStream output = mapOutput.getOutputStream();
output.write(HEADER);
output.write(buffer);
output.write(Ints.toByteArray((int) ChecksumUtils.getCrc32(buffer)));
diff --git
a/client-tez/src/main/java/org/apache/tez/runtime/library/input/RssUnorderedKVInput.java
b/client-tez/src/main/java/org/apache/tez/runtime/library/input/RssUnorderedKVInput.java
index 3cdc9c9e0..f9d03c712 100644
---
a/client-tez/src/main/java/org/apache/tez/runtime/library/input/RssUnorderedKVInput.java
+++
b/client-tez/src/main/java/org/apache/tez/runtime/library/input/RssUnorderedKVInput.java
@@ -185,7 +185,8 @@ public class RssUnorderedKVInput extends
AbstractLogicalInput {
getContext().getDagIdentifier(),
conf,
getContext().getTotalMemoryAvailableToTask(),
- memoryUpdateCallbackHandler.getMemoryAssigned());
+ memoryUpdateCallbackHandler.getMemoryAssigned(),
+ applicationAttemptId.toString());
this.rssShuffleManager =
new RssShuffleManager(
diff --git
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocatorTest.java
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocatorTest.java
index ab822f233..af501fd79 100644
---
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocatorTest.java
+++
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssSimpleFetchedInputAllocatorTest.java
@@ -17,20 +17,20 @@
package org.apache.tez.runtime.library.common.shuffle.impl;
+import java.io.File;
import java.io.IOException;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
-import org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput;
import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
import org.junit.jupiter.api.Test;
-import org.mockito.MockedConstruction;
-import org.mockito.Mockito;
+import org.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.tez.common.TezRuntimeFrameworkConfigs.LOCAL_DIRS;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class RssSimpleFetchedInputAllocatorTest {
@@ -39,8 +39,9 @@ public class RssSimpleFetchedInputAllocatorTest {
LoggerFactory.getLogger(RssSimpleFetchedInputAllocatorTest.class);
@Test
- public void testAllocate() throws IOException {
+ public void testAllocate(@TempDir File tmpDir) throws IOException {
Configuration conf = new Configuration();
+ conf.set(LOCAL_DIRS, tmpDir + "/local");
long jvmMax = 954728448L;
LOG.info("jvmMax: " + jvmMax);
@@ -59,7 +60,8 @@ public class RssSimpleFetchedInputAllocatorTest {
123,
conf,
Runtime.getRuntime().maxMemory(),
- inMemThreshold);
+ inMemThreshold,
+ "");
long requestSize = (long) (0.4f * inMemThreshold);
long compressedSize = 1L;
@@ -73,31 +75,23 @@ public class RssSimpleFetchedInputAllocatorTest {
inputManager.allocate(requestSize, compressedSize, new
InputAttemptIdentifier(2, 1));
assertEquals(FetchedInput.Type.MEMORY, fi2.getType());
- MockedConstruction<DiskFetchedInput> mockedConstruction =
- Mockito.mockConstruction(
- DiskFetchedInput.class,
- ((mockedInput, context) -> {
- // Over limit by this point. Next reserve should give back a
DISK allocation
- FetchedInput fi3 =
- inputManager.allocate(
- requestSize, compressedSize, new
InputAttemptIdentifier(3, 1));
- assertEquals(FetchedInput.Type.DISK, fi3.getType());
-
- // Freed one memory allocation. Next should be mem again.
- fi1.abort();
- fi1.free();
- FetchedInput fi4 =
- inputManager.allocate(
- requestSize, compressedSize, new
InputAttemptIdentifier(4, 1));
- assertEquals(FetchedInput.Type.MEMORY, fi4.getType());
-
- // Freed one disk allocation. Next sould be disk again (no mem
freed)
- fi3.abort();
- fi3.free();
- FetchedInput fi5 =
- inputManager.allocate(
- requestSize, compressedSize, new
InputAttemptIdentifier(4, 1));
- assertEquals(FetchedInput.Type.DISK, fi5.getType());
- }));
+ // Over limit by this point. Next reserve should give back a DISK
allocation
+ FetchedInput fi3 =
+ inputManager.allocate(requestSize, compressedSize, new
InputAttemptIdentifier(3, 1));
+ assertEquals(FetchedInput.Type.DISK, fi3.getType());
+
+ // Freed one memory allocation. Next should be mem again.
+ fi1.abort();
+ fi1.free();
+ FetchedInput fi4 =
+ inputManager.allocate(requestSize, compressedSize, new
InputAttemptIdentifier(4, 1));
+ assertEquals(FetchedInput.Type.MEMORY, fi4.getType());
+
+ // Freed one disk allocation. Next sould be disk again (no mem freed)
+ fi3.abort();
+ fi3.free();
+ FetchedInput fi5 =
+ inputManager.allocate(requestSize, compressedSize, new
InputAttemptIdentifier(4, 1));
+ assertEquals(FetchedInput.Type.DISK, fi5.getType());
}
}
diff --git
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssTezFetcherTest.java
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssTezFetcherTest.java
index c9bfca61c..c36c2f297 100644
---
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssTezFetcherTest.java
+++
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/RssTezFetcherTest.java
@@ -17,6 +17,7 @@
package org.apache.tez.runtime.library.common.shuffle.impl;
+import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
@@ -26,6 +27,8 @@ import java.util.Random;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.io.BoundedByteArrayOutputStream;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.DataOutputBuffer;
@@ -35,6 +38,7 @@ import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.readers.UnorderedKVReader;
import org.apache.tez.runtime.library.common.shuffle.FetchedInput;
import org.apache.tez.runtime.library.common.shuffle.FetcherCallback;
import org.apache.tez.runtime.library.common.shuffle.MemoryFetchedInput;
@@ -42,7 +46,10 @@ import
org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryRead
import
org.apache.tez.runtime.library.common.shuffle.orderedgrouped.InMemoryWriter;
import org.apache.tez.runtime.library.common.sort.impl.IFile;
import org.apache.tez.runtime.library.utils.BufferUtils;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -52,6 +59,12 @@ import org.apache.uniffle.common.compression.Codec;
import org.apache.uniffle.common.compression.Lz4Codec;
import org.apache.uniffle.common.config.RssConf;
+import static
org.apache.tez.common.RssTezConfig.RSS_REDUCE_REMOTE_SPILL_ENABLED;
+import static org.apache.tez.common.RssTezConfig.RSS_REMOTE_SPILL_STORAGE_PATH;
+import static org.apache.tez.common.TezRuntimeFrameworkConfigs.LOCAL_DIRS;
+import static
org.apache.tez.runtime.library.api.TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS;
+import static
org.apache.tez.runtime.library.api.TezRuntimeConfiguration.TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT;
+import static
org.apache.tez.runtime.library.api.TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.fail;
@@ -65,6 +78,25 @@ public class RssTezFetcherTest {
static List<KVPair> textData;
static Codec codec = new Lz4Codec();
+ private static FileSystem remoteFS;
+ private static MiniDFSCluster cluster;
+
+ @BeforeAll
+ public static void setUpHdfs(@TempDir File tempDir) throws Exception {
+ Configuration conf = new Configuration();
+ File baseDir = tempDir;
+ conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+ cluster = (new MiniDFSCluster.Builder(conf)).build();
+ String hdfsUri = cluster.getURI().toString() + "/";
+ remoteFS = (new Path(hdfsUri)).getFileSystem(conf);
+ }
+
+ @AfterAll
+ public static void tearDownHdfs() throws Exception {
+ remoteFS.close();
+ cluster.shutdown();
+ }
+
@Test
public void writeAndReadDataTestWithoutRss() throws Throwable {
fs = FileSystem.getLocal(conf);
@@ -141,6 +173,149 @@ public class RssTezFetcherTest {
}
}
+ @Test
+ public void testReadWithDiskFetchedInput(@TempDir File tmpDir) throws
Throwable {
+ initRssData();
+ conf.setFloat(TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 0.25F);
+ conf.set(LOCAL_DIRS, tmpDir + "/local");
+ conf.set(TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+ conf.set(TEZ_RUNTIME_VALUE_CLASS, IntWritable.class.getName());
+ RssSimpleFetchedInputAllocator inputManager =
+ new RssSimpleFetchedInputAllocator(
+ "test", "uniqueId1", 0, conf, 2 * 1024 * 1024, 2 * 1024 * 1024,
"appattemptid1");
+ List<FetchedInput> inputs = new ArrayList<>();
+ FetcherCallback fetcherCallback =
+ new FetcherCallback() {
+ @Override
+ public void fetchSucceeded(
+ String host,
+ InputAttemptIdentifier srcAttemptIdentifier,
+ FetchedInput fetchedInput,
+ long fetchedBytes,
+ long decompressedLength,
+ long copyDuration)
+ throws IOException {
+ LOG.info("Fetch success");
+ fetchedInput.commit();
+ inputs.add(fetchedInput);
+ }
+
+ @Override
+ public void fetchFailed(
+ String host, InputAttemptIdentifier srcAttemptIdentifier,
boolean connectFailed) {
+ fail();
+ }
+ };
+
+ ShuffleReadClient shuffleReadClient = new MockedShuffleReadClient(data);
+ RssTezFetcher rssFetcher =
+ new RssTezFetcher(fetcherCallback, inputManager, shuffleReadClient,
null, 2, new RssConf());
+ rssFetcher.fetchAllRssBlocks();
+ for (int i = 0; i < data.size(); i++) {
+ Text readKey = new Text();
+ IntWritable readValue = new IntWritable();
+ Deserializer<Text> keyDeserializer;
+ Deserializer<IntWritable> valDeserializer;
+ SerializationFactory serializationFactory = new
SerializationFactory(conf);
+ keyDeserializer = serializationFactory.getDeserializer(Text.class);
+ valDeserializer =
serializationFactory.getDeserializer(IntWritable.class);
+ DataInputBuffer keyIn = new DataInputBuffer();
+ DataInputBuffer valIn = new DataInputBuffer();
+ keyDeserializer.open(keyIn);
+ valDeserializer.open(valIn);
+
+ UnorderedKVReader unorderedKVReader =
+ new UnorderedKVReader(null, conf, null, false, 0, 0, null, null);
+ IFile.Reader reader = unorderedKVReader.openIFileReader(inputs.get(i));
+ int numRecordsRead = 0;
+ while (reader.nextRawKey(keyIn)) {
+ reader.nextRawValue(valIn);
+ readKey = keyDeserializer.deserialize(readKey);
+ readValue = valDeserializer.deserialize(readValue);
+
+ KVPair expected = textData.get(numRecordsRead);
+ assertEquals(expected.getKey(), readKey);
+ assertEquals(expected.getvalue(), readValue);
+
+ numRecordsRead++;
+ }
+ assertEquals(textData.size(), numRecordsRead);
+ }
+ }
+
+ @Test
+ public void testReadWithRemoteFetchedInput(@TempDir File tmpDir) throws
Throwable {
+ initRssData();
+ conf.setFloat(TEZ_RUNTIME_SHUFFLE_MEMORY_LIMIT_PERCENT, 0.25F);
+ conf.set(LOCAL_DIRS, tmpDir + "/local");
+ conf.set(TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+ conf.set(TEZ_RUNTIME_VALUE_CLASS, IntWritable.class.getName());
+ conf.setBoolean(RSS_REDUCE_REMOTE_SPILL_ENABLED, true);
+ conf.set(RSS_REMOTE_SPILL_STORAGE_PATH, "/tmp/spill");
+ RssSimpleFetchedInputAllocator inputManager =
+ new RssSimpleFetchedInputAllocator(
+ "test", "uniqueId1", 0, conf, 2 * 1024 * 1024, 2 * 1024 * 1024,
"appattemptid1");
+ inputManager.setRemoteFS(remoteFS);
+ List<FetchedInput> inputs = new ArrayList<>();
+ FetcherCallback fetcherCallback =
+ new FetcherCallback() {
+ @Override
+ public void fetchSucceeded(
+ String host,
+ InputAttemptIdentifier srcAttemptIdentifier,
+ FetchedInput fetchedInput,
+ long fetchedBytes,
+ long decompressedLength,
+ long copyDuration)
+ throws IOException {
+ LOG.info("Fetch success");
+ fetchedInput.commit();
+ inputs.add(fetchedInput);
+ }
+
+ @Override
+ public void fetchFailed(
+ String host, InputAttemptIdentifier srcAttemptIdentifier,
boolean connectFailed) {
+ fail();
+ }
+ };
+
+ ShuffleReadClient shuffleReadClient = new MockedShuffleReadClient(data);
+ RssTezFetcher rssFetcher =
+ new RssTezFetcher(fetcherCallback, inputManager, shuffleReadClient,
null, 2, new RssConf());
+ rssFetcher.fetchAllRssBlocks();
+ for (int i = 0; i < data.size(); i++) {
+ Text readKey = new Text();
+ IntWritable readValue = new IntWritable();
+ Deserializer<Text> keyDeserializer;
+ Deserializer<IntWritable> valDeserializer;
+ SerializationFactory serializationFactory = new
SerializationFactory(conf);
+ keyDeserializer = serializationFactory.getDeserializer(Text.class);
+ valDeserializer =
serializationFactory.getDeserializer(IntWritable.class);
+ DataInputBuffer keyIn = new DataInputBuffer();
+ DataInputBuffer valIn = new DataInputBuffer();
+ keyDeserializer.open(keyIn);
+ valDeserializer.open(valIn);
+
+ UnorderedKVReader unorderedKVReader =
+ new UnorderedKVReader(null, conf, null, false, 0, 0, null, null);
+ IFile.Reader reader = unorderedKVReader.openIFileReader(inputs.get(i));
+ int numRecordsRead = 0;
+ while (reader.nextRawKey(keyIn)) {
+ reader.nextRawValue(valIn);
+ readKey = keyDeserializer.deserialize(readKey);
+ readValue = valDeserializer.deserialize(readValue);
+
+ KVPair expected = textData.get(numRecordsRead);
+ assertEquals(expected.getKey(), readKey);
+ assertEquals(expected.getvalue(), readValue);
+
+ numRecordsRead++;
+ }
+ assertEquals(textData.size(), numRecordsRead);
+ }
+ }
+
private static void initRssData() throws Exception {
InMemoryWriter writer = null;
BoundedByteArrayOutputStream bout = new BoundedByteArrayOutputStream(1024
* 1024);
diff --git
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriterTest.java
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriterTest.java
index 237a66582..315f9d26b 100644
---
a/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriterTest.java
+++
b/client-tez/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/RssTezBypassWriterTest.java
@@ -17,22 +17,60 @@
package org.apache.tez.runtime.library.common.shuffle.orderedgrouped;
+import java.io.File;
import java.io.IOException;
+import java.io.InputStream;
import java.util.Arrays;
import com.google.common.primitives.Ints;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.common.shuffle.DiskFetchedInput;
+import org.apache.tez.runtime.library.common.shuffle.FetchedInputCallback;
+import org.apache.tez.runtime.library.common.shuffle.RemoteFetchedInput;
+import
org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import org.apache.uniffle.common.util.ChecksumUtils;
+import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.ArgumentMatchers.anyInt;
+import static org.mockito.ArgumentMatchers.anyLong;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
public class RssTezBypassWriterTest {
+
+ private static FileSystem remoteFS;
+ private static MiniDFSCluster cluster;
+
+ @BeforeAll
+ public static void setUpHdfs(@TempDir File tempDir) throws Exception {
+ Configuration conf = new Configuration();
+ File baseDir = tempDir;
+ conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+ cluster = (new MiniDFSCluster.Builder(conf)).build();
+ String hdfsUri = cluster.getURI().toString() + "/";
+ remoteFS = (new Path(hdfsUri)).getFileSystem(conf);
+ }
+
+ @AfterAll
+ public static void tearDownHdfs() throws Exception {
+ remoteFS.close();
+ cluster.shutdown();
+ }
+
@Test
public void testWrite() {
byte[] data = new byte[] {1, 2, -1, 1, 2, -1, -1};
- ;
MapOutput mapOutput = MapOutput.createMemoryMapOutput(null, null, 7, true);
RssTezBypassWriter.write(mapOutput, data);
byte[] r = mapOutput.getMemory();
@@ -49,4 +87,72 @@ public class RssTezBypassWriterTest {
byte[] result = new byte[] {-71, -87, 19, -71};
assertTrue(Arrays.equals(Ints.toByteArray((int)
ChecksumUtils.getCrc32(data)), result));
}
+
+ @Test
+ public void testWriteDiskFetchInput(@TempDir File tmpDir) throws IOException
{
+ byte[] data = new byte[] {1, 2, -1, 1, 2, -1, -1};
+ InputAttemptIdentifier inputAttemptIdentifier = new
InputAttemptIdentifier(1, 1);
+ TezTaskOutputFiles tezTaskOutputFiles = mock(TezTaskOutputFiles.class);
+ Path output = new Path(tmpDir.toString(), "out");
+ when(tezTaskOutputFiles.getInputFileForWrite(anyInt(), anyInt(),
anyLong())).thenReturn(output);
+ FetchedInputCallback callback = mock(FetchedInputCallback.class);
+ Configuration conf = new Configuration();
+ DiskFetchedInput fetchedInput =
+ new DiskFetchedInput(
+ data.length + 8, inputAttemptIdentifier, callback, conf, null,
tezTaskOutputFiles);
+ RssTezBypassWriter.write(fetchedInput, data);
+
+ // Verify result
+ FileSystem localFS = FileSystem.getLocal(conf).getRaw();
+ fetchedInput.commit();
+ assertEquals(output, fetchedInput.getInputPath());
+ assertTrue(localFS.exists(fetchedInput.getInputPath()));
+ assertEquals(data.length + 8, fetchedInput.getSize());
+ assertEquals(data.length + 8,
localFS.getFileStatus(fetchedInput.getInputPath()).getLen());
+ InputStream inputStream = fetchedInput.getInputStream();
+ byte[] out = new byte[data.length + 8];
+ assertEquals(data.length + 8, inputStream.read(out, 0, data.length + 8));
+ // Ignore the first four and last four characters
+ for (int i = 4; i < data.length + 4; i++) {
+ assertEquals(data[i - 4], out[i]);
+ }
+ inputStream.close();
+ fetchedInput.free();
+ assertFalse(localFS.exists(fetchedInput.getInputPath()));
+ }
+
+ @Test
+ public void testWriteRemoteFetchInput() throws IOException {
+ byte[] data = new byte[] {1, 2, -1, 1, 2, -1, -1};
+ InputAttemptIdentifier inputAttemptIdentifier = new
InputAttemptIdentifier(1, 1);
+ FetchedInputCallback callback = mock(FetchedInputCallback.class);
+ RemoteFetchedInput fetchedInput =
+ new RemoteFetchedInput(
+ data.length + 8,
+ inputAttemptIdentifier,
+ callback,
+ remoteFS,
+ "/base",
+ "uniqueid1",
+ "appattemptid1");
+ RssTezBypassWriter.write(fetchedInput, data);
+
+ // Verify result
+ fetchedInput.commit();
+ assertEquals(
+ "/base/appattemptid1/uniqueid1_src_1_spill_-1.out",
fetchedInput.getInputPath().toString());
+ assertTrue(remoteFS.exists(fetchedInput.getInputPath()));
+ assertEquals(data.length + 8, fetchedInput.getSize());
+ assertEquals(data.length + 8,
remoteFS.getFileStatus(fetchedInput.getInputPath()).getLen());
+ InputStream inputStream = fetchedInput.getInputStream();
+ byte[] out = new byte[data.length + 8];
+ assertEquals(data.length + 8, inputStream.read(out, 0, data.length + 8));
+ // Ignore the first four and last four characters
+ for (int i = 4; i < data.length + 4; i++) {
+ assertEquals(data[i - 4], out[i]);
+ }
+ inputStream.close();
+ fetchedInput.free();
+ assertFalse(remoteFS.exists(fetchedInput.getInputPath()));
+ }
}
diff --git
a/common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java
b/common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java
index 0bcc09be8..093bc7bd2 100644
--- a/common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java
+++ b/common/src/main/java/org/apache/uniffle/common/RemoteStorageInfo.java
@@ -49,18 +49,23 @@ public class RemoteStorageInfo implements Serializable {
public RemoteStorageInfo(String path, String confString) {
this.path = path;
- this.confItems = Maps.newHashMap();
+ this.confItems = parseRemoteStorageConf(confString);
+ }
+
+ public static Map<String, String> parseRemoteStorageConf(String confString) {
+ Map<String, String> confItems = Maps.newHashMap();
if (!StringUtils.isEmpty(confString)) {
String[] items = confString.split(Constants.COMMA_SPLIT_CHAR);
if (!ArrayUtils.isEmpty(items)) {
for (String item : items) {
String[] kv = item.split(Constants.EQUAL_SPLIT_CHAR);
if (kv.length == 2) {
- this.confItems.put(kv[0], kv[1]);
+ confItems.put(kv[0], kv[1]);
}
}
}
}
+ return confItems;
}
public String getPath() {