|
@@ -42,6 +42,7 @@ import org.apache.hadoop.io.Text;
|
|
import org.apache.hadoop.mapreduce.Mapper;
|
|
import org.apache.hadoop.mapreduce.Mapper;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
|
+import org.apache.hadoop.tools.CopyListingFileStatus;
|
|
import org.apache.hadoop.tools.DistCpConstants;
|
|
import org.apache.hadoop.tools.DistCpConstants;
|
|
import org.apache.hadoop.tools.DistCpOptionSwitch;
|
|
import org.apache.hadoop.tools.DistCpOptionSwitch;
|
|
import org.apache.hadoop.tools.DistCpOptions;
|
|
import org.apache.hadoop.tools.DistCpOptions;
|
|
@@ -222,7 +223,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -238,7 +239,7 @@ public class TestCopyMapper {
|
|
|
|
|
|
for (Path path: pathList) {
|
|
for (Path path: pathList) {
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fs.getFileStatus(path), context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fs.getFileStatus(path)), context);
|
|
}
|
|
}
|
|
|
|
|
|
// Check that the maps worked.
|
|
// Check that the maps worked.
|
|
@@ -283,12 +284,11 @@ public class TestCopyMapper {
|
|
}
|
|
}
|
|
|
|
|
|
private void testCopyingExistingFiles(FileSystem fs, CopyMapper copyMapper,
|
|
private void testCopyingExistingFiles(FileSystem fs, CopyMapper copyMapper,
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context) {
|
|
|
|
-
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context) {
|
|
try {
|
|
try {
|
|
for (Path path : pathList) {
|
|
for (Path path : pathList) {
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fs.getFileStatus(path), context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fs.getFileStatus(path)), context);
|
|
}
|
|
}
|
|
|
|
|
|
Assert.assertEquals(nFiles,
|
|
Assert.assertEquals(nFiles,
|
|
@@ -309,7 +309,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -320,7 +320,7 @@ public class TestCopyMapper {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
|
|
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), pathList.get(0))),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), pathList.get(0))),
|
|
- fs.getFileStatus(pathList.get(0)), context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fs.getFileStatus(pathList.get(0))), context);
|
|
|
|
|
|
Assert.assertTrue("There should have been an exception.", false);
|
|
Assert.assertTrue("There should have been an exception.", false);
|
|
}
|
|
}
|
|
@@ -343,7 +343,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
mkdirs(SOURCE_PATH + "/src/file");
|
|
mkdirs(SOURCE_PATH + "/src/file");
|
|
@@ -351,7 +351,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- fs.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(fs.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
|
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
|
@@ -372,22 +373,24 @@ public class TestCopyMapper {
|
|
|
|
|
|
final CopyMapper copyMapper = new CopyMapper();
|
|
final CopyMapper copyMapper = new CopyMapper();
|
|
|
|
|
|
- final Mapper<Text, FileStatus, Text, Text>.Context context = tmpUser.
|
|
|
|
- doAs(new PrivilegedAction<Mapper<Text, FileStatus, Text, Text>.Context>() {
|
|
|
|
- @Override
|
|
|
|
- public Mapper<Text, FileStatus, Text, Text>.Context run() {
|
|
|
|
- try {
|
|
|
|
- StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
|
|
- return stubContext.getContext();
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- LOG.error("Exception encountered ", e);
|
|
|
|
- throw new RuntimeException(e);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
|
|
+ final Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
|
|
|
|
+ tmpUser.doAs(
|
|
|
|
+ new PrivilegedAction<Mapper<Text, CopyListingFileStatus, Text, Text>.Context>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Mapper<Text, CopyListingFileStatus, Text, Text>.Context run() {
|
|
|
|
+ try {
|
|
|
|
+ StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
|
|
+ return stubContext.getContext();
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Exception encountered ", e);
|
|
|
|
+ throw new RuntimeException(e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
|
|
+ preserveStatus.remove(DistCpOptions.FileAttribute.ACL);
|
|
|
|
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
DistCpUtils.packAttributes(preserveStatus));
|
|
DistCpUtils.packAttributes(preserveStatus));
|
|
@@ -415,7 +418,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(tmpFS.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
Assert.fail("Expected copy to fail");
|
|
Assert.fail("Expected copy to fail");
|
|
} catch (AccessControlException e) {
|
|
} catch (AccessControlException e) {
|
|
@@ -442,19 +446,20 @@ public class TestCopyMapper {
|
|
|
|
|
|
final CopyMapper copyMapper = new CopyMapper();
|
|
final CopyMapper copyMapper = new CopyMapper();
|
|
|
|
|
|
- final Mapper<Text, FileStatus, Text, Text>.Context context = tmpUser.
|
|
|
|
- doAs(new PrivilegedAction<Mapper<Text, FileStatus, Text, Text>.Context>() {
|
|
|
|
- @Override
|
|
|
|
- public Mapper<Text, FileStatus, Text, Text>.Context run() {
|
|
|
|
- try {
|
|
|
|
- StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
|
|
- return stubContext.getContext();
|
|
|
|
- } catch (Exception e) {
|
|
|
|
- LOG.error("Exception encountered ", e);
|
|
|
|
- throw new RuntimeException(e);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
|
|
+ final Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
|
|
|
|
+ tmpUser.doAs(
|
|
|
|
+ new PrivilegedAction<Mapper<Text, CopyListingFileStatus, Text, Text>.Context>() {
|
|
|
|
+ @Override
|
|
|
|
+ public Mapper<Text, CopyListingFileStatus, Text, Text>.Context run() {
|
|
|
|
+ try {
|
|
|
|
+ StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
|
|
+ return stubContext.getContext();
|
|
|
|
+ } catch (Exception e) {
|
|
|
|
+ LOG.error("Exception encountered ", e);
|
|
|
|
+ throw new RuntimeException(e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ });
|
|
|
|
|
|
touchFile(SOURCE_PATH + "/src/file");
|
|
touchFile(SOURCE_PATH + "/src/file");
|
|
mkdirs(TARGET_PATH);
|
|
mkdirs(TARGET_PATH);
|
|
@@ -481,7 +486,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(tmpFS.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
} catch (Exception e) {
|
|
} catch (Exception e) {
|
|
throw new RuntimeException(e);
|
|
throw new RuntimeException(e);
|
|
@@ -518,9 +524,11 @@ public class TestCopyMapper {
|
|
}
|
|
}
|
|
});
|
|
});
|
|
|
|
|
|
- final Mapper<Text, FileStatus, Text, Text>.Context context = stubContext.getContext();
|
|
|
|
|
|
+ final Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
|
|
|
|
+ stubContext.getContext();
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
|
|
+ preserveStatus.remove(DistCpOptions.FileAttribute.ACL);
|
|
|
|
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
DistCpUtils.packAttributes(preserveStatus));
|
|
DistCpUtils.packAttributes(preserveStatus));
|
|
@@ -551,7 +559,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(tmpFS.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
Assert.assertEquals(stubContext.getWriter().values().size(), 1);
|
|
Assert.assertEquals(stubContext.getWriter().values().size(), 1);
|
|
Assert.assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP"));
|
|
Assert.assertTrue(stubContext.getWriter().values().get(0).toString().startsWith("SKIP"));
|
|
@@ -594,8 +603,9 @@ public class TestCopyMapper {
|
|
|
|
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet<DistCpOptions.FileAttribute> preserveStatus =
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
EnumSet.allOf(DistCpOptions.FileAttribute.class);
|
|
|
|
+ preserveStatus.remove(DistCpOptions.FileAttribute.ACL);
|
|
|
|
|
|
- final Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ final Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
context.getConfiguration().set(DistCpConstants.CONF_LABEL_PRESERVE_STATUS,
|
|
@@ -629,7 +639,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- tmpFS.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(tmpFS.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
Assert.fail("Didn't expect the file to be copied");
|
|
Assert.fail("Didn't expect the file to be copied");
|
|
} catch (AccessControlException ignore) {
|
|
} catch (AccessControlException ignore) {
|
|
@@ -661,7 +672,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
touchFile(SOURCE_PATH + "/src/file");
|
|
touchFile(SOURCE_PATH + "/src/file");
|
|
@@ -669,7 +680,8 @@ public class TestCopyMapper {
|
|
try {
|
|
try {
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
copyMapper.map(new Text("/src/file"),
|
|
copyMapper.map(new Text("/src/file"),
|
|
- fs.getFileStatus(new Path(SOURCE_PATH + "/src/file")),
|
|
|
|
|
|
+ new CopyListingFileStatus(fs.getFileStatus(
|
|
|
|
+ new Path(SOURCE_PATH + "/src/file"))),
|
|
context);
|
|
context);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
|
Assert.assertTrue(e.getMessage().startsWith("Can't replace"));
|
|
@@ -688,7 +700,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -705,7 +717,7 @@ public class TestCopyMapper {
|
|
if (!fileStatus.isDirectory()) {
|
|
if (!fileStatus.isDirectory()) {
|
|
fs.delete(path, true);
|
|
fs.delete(path, true);
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fileStatus, context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fileStatus), context);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
if (ignoreFailures) {
|
|
if (ignoreFailures) {
|
|
@@ -745,7 +757,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -759,7 +771,7 @@ public class TestCopyMapper {
|
|
for (Path path : pathList) {
|
|
for (Path path : pathList) {
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fileStatus, context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fileStatus), context);
|
|
}
|
|
}
|
|
|
|
|
|
Assert.fail("Copy should have failed because of block-size difference.");
|
|
Assert.fail("Copy should have failed because of block-size difference.");
|
|
@@ -780,7 +792,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -798,7 +810,7 @@ public class TestCopyMapper {
|
|
for (Path path : pathList) {
|
|
for (Path path : pathList) {
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fileStatus, context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fileStatus), context);
|
|
}
|
|
}
|
|
|
|
|
|
// Check that the block-size/replication aren't preserved.
|
|
// Check that the block-size/replication aren't preserved.
|
|
@@ -855,7 +867,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
context.getConfiguration().set(
|
|
context.getConfiguration().set(
|
|
@@ -863,7 +875,8 @@ public class TestCopyMapper {
|
|
targetFilePath.getParent().toString()); // Parent directory.
|
|
targetFilePath.getParent().toString()); // Parent directory.
|
|
copyMapper.setup(context);
|
|
copyMapper.setup(context);
|
|
|
|
|
|
- final FileStatus sourceFileStatus = fs.getFileStatus(sourceFilePath);
|
|
|
|
|
|
+ final CopyListingFileStatus sourceFileStatus = new CopyListingFileStatus(
|
|
|
|
+ fs.getFileStatus(sourceFilePath));
|
|
|
|
|
|
long before = fs.getFileStatus(targetFilePath).getModificationTime();
|
|
long before = fs.getFileStatus(targetFilePath).getModificationTime();
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(
|
|
@@ -907,7 +920,7 @@ public class TestCopyMapper {
|
|
FileSystem fs = cluster.getFileSystem();
|
|
FileSystem fs = cluster.getFileSystem();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
CopyMapper copyMapper = new CopyMapper();
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
|
- Mapper<Text, FileStatus, Text, Text>.Context context
|
|
|
|
|
|
+ Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
|
= stubContext.getContext();
|
|
= stubContext.getContext();
|
|
|
|
|
|
Configuration configuration = context.getConfiguration();
|
|
Configuration configuration = context.getConfiguration();
|
|
@@ -926,7 +939,7 @@ public class TestCopyMapper {
|
|
for (Path path : pathList) {
|
|
for (Path path : pathList) {
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
final FileStatus fileStatus = fs.getFileStatus(path);
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
|
- fileStatus, context);
|
|
|
|
|
|
+ new CopyListingFileStatus(fileStatus), context);
|
|
}
|
|
}
|
|
|
|
|
|
// Check that the user/group attributes are preserved
|
|
// Check that the user/group attributes are preserved
|