HADOOP-16018. DistCp won't reassemble chunks when blocks per chunk > 0.
Contributed by Kai Xie.
(cherry picked from commit 188bebbe7e
)
This commit is contained in:
parent
1dc2b49bfd
commit
5dce9d75e6
|
@ -123,6 +123,10 @@ public final class DistCpConstants {
|
|||
public static final String CONF_LABEL_COPY_BUFFER_SIZE =
|
||||
"distcp.copy.buffer.size";
|
||||
|
||||
/** DistCp Blocks Per Chunk: {@value}. */
|
||||
public static final String CONF_LABEL_BLOCKS_PER_CHUNK =
|
||||
"distcp.blocks.per.chunk";
|
||||
|
||||
/**
|
||||
* Constants for DistCp return code to shell / consumer of ToolRunner's run
|
||||
*/
|
||||
|
|
|
@ -192,7 +192,7 @@ public enum DistCpOptionSwitch {
|
|||
new Option("sizelimit", true, "(Deprecated!) Limit number of files " +
|
||||
"copied to <= n bytes")),
|
||||
|
||||
BLOCKS_PER_CHUNK("",
|
||||
BLOCKS_PER_CHUNK(DistCpConstants.CONF_LABEL_BLOCKS_PER_CHUNK,
|
||||
new Option("blocksperchunk", true, "If set to a positive value, files"
|
||||
+ "with more blocks than this value will be split into chunks of "
|
||||
+ "<blocksperchunk> blocks to be transferred in parallel, and "
|
||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.tools;
|
|||
|
||||
import java.util.Collections;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
|
@ -533,4 +534,32 @@ public class TestDistCpOptions {
|
|||
builder.withLogPath(logPath).withVerboseLog(true);
|
||||
Assert.assertTrue(builder.build().shouldVerboseLog());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendToConf() {
|
||||
final int expectedBlocksPerChunk = 999;
|
||||
final String expectedValForEmptyConfigKey = "VALUE_OF_EMPTY_CONFIG_KEY";
|
||||
|
||||
DistCpOptions options = new DistCpOptions.Builder(
|
||||
Collections.singletonList(
|
||||
new Path("hdfs://localhost:8020/source")),
|
||||
new Path("hdfs://localhost:8020/target/"))
|
||||
.withBlocksPerChunk(expectedBlocksPerChunk)
|
||||
.build();
|
||||
|
||||
Configuration config = new Configuration();
|
||||
config.set("", expectedValForEmptyConfigKey);
|
||||
|
||||
options.appendToConf(config);
|
||||
Assert.assertEquals(expectedBlocksPerChunk,
|
||||
config.getInt(
|
||||
DistCpOptionSwitch
|
||||
.BLOCKS_PER_CHUNK
|
||||
.getConfigLabel(), 0));
|
||||
Assert.assertEquals(
|
||||
"Some DistCpOptionSwitch's config label is empty! " +
|
||||
"Pls ensure the config label is provided when apply to config, " +
|
||||
"otherwise it may not be fetched properly",
|
||||
expectedValForEmptyConfigKey, config.get(""));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue