HBASE-26707: Reduce number of renames during bulkload (#4066) (#4122)

Signed-off-by: Wellington Ramos Chevreuil <wchevreuil@apache.org>
This commit is contained in:
BukrosSzabolcs 2022-02-25 21:11:41 +01:00 committed by GitHub
parent 82282a9f95
commit 77bb153a2e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 525 additions and 37 deletions

View File

@ -0,0 +1,108 @@
/**
*
* 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.hadoop.hbase.mapreduce;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.IntegrationTestingUtility;
import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
import org.apache.hadoop.hbase.testclassification.IntegrationTests;
import org.apache.hadoop.util.ToolRunner;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Test Bulk Load and MR on a distributed cluster.
* With FileBased StorefileTracker enabled.
* It starts an MR job that creates linked chains
*
* The format of rows is like this:
* Row Key -> Long
*
* L:<< Chain Id >> -> Row Key of the next link in the chain
* S:<< Chain Id >> -> The step in the chain that his link is.
* D:<< Chain Id >> -> Random Data.
*
* All chains start on row 0.
* All rk's are > 0.
*
* After creating the linked lists they are walked over using a TableMapper based Mapreduce Job.
*
* There are a few options exposed:
*
* hbase.IntegrationTestBulkLoad.chainLength
* The number of rows that will be part of each and every chain.
*
* hbase.IntegrationTestBulkLoad.numMaps
* The number of mappers that will be run. Each mapper creates on linked list chain.
*
* hbase.IntegrationTestBulkLoad.numImportRounds
* How many jobs will be run to create linked lists.
*
* hbase.IntegrationTestBulkLoad.tableName
* The name of the table.
*
* hbase.IntegrationTestBulkLoad.replicaCount
* How many region replicas to configure for the table under test.
*/
@Category(IntegrationTests.class)
public class IntegrationTestFileBasedSFTBulkLoad extends IntegrationTestBulkLoad {
private static final Logger LOG = LoggerFactory.getLogger(IntegrationTestFileBasedSFTBulkLoad.class);
private static String NUM_MAPS_KEY = "hbase.IntegrationTestBulkLoad.numMaps";
private static String NUM_IMPORT_ROUNDS_KEY = "hbase.IntegrationTestBulkLoad.numImportRounds";
private static String NUM_REPLICA_COUNT_KEY = "hbase.IntegrationTestBulkLoad.replicaCount";
private static int NUM_REPLICA_COUNT_DEFAULT = 1;
@Test
public void testFileBasedSFTBulkLoad() throws Exception {
super.testBulkLoad();
}
@Override
public void setUpCluster() throws Exception {
util = getTestingUtil(getConf());
util.getConfiguration().set(StoreFileTrackerFactory.TRACKER_IMPL,
"org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
util.initializeCluster(1);
int replicaCount = getConf().getInt(NUM_REPLICA_COUNT_KEY, NUM_REPLICA_COUNT_DEFAULT);
if (LOG.isDebugEnabled() && replicaCount != NUM_REPLICA_COUNT_DEFAULT) {
LOG.debug("Region Replicas enabled: " + replicaCount);
}
// Scale this up on a real cluster
if (util.isDistributedCluster()) {
util.getConfiguration().setIfUnset(NUM_MAPS_KEY,
Integer.toString(util.getAdmin().getRegionServers().size() * 10));
util.getConfiguration().setIfUnset(NUM_IMPORT_ROUNDS_KEY, "5");
} else {
util.startMiniMapReduceCluster();
}
}
public static void main(String[] args) throws Exception {
Configuration conf = HBaseConfiguration.create();
IntegrationTestingUtility.setUseDistributedCluster(conf);
int status = ToolRunner.run(conf, new IntegrationTestFileBasedSFTBulkLoad(), args);
System.exit(status);
}
}

View File

@ -6844,7 +6844,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return final path to be used for actual loading
* @throws IOException
*/
String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile)
String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile, String customStaging)
throws IOException;
/**
@ -6966,12 +6966,21 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
familyWithFinalPath.put(familyName, new ArrayList<>());
}
List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
String finalPath = path;
try {
String finalPath = path;
boolean reqTmp = store.storeEngine.requireWritingToTmpDirFirst();
if (bulkLoadListener != null) {
finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile);
finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile,
reqTmp ? null : regionDir.toString());
}
Pair<Path, Path> pair = null;
if (reqTmp) {
pair = store.preBulkLoadHFile(finalPath, seqId);
}
else {
Path livePath = new Path(finalPath);
pair = new Pair<>(livePath, livePath);
}
Pair<Path, Path> pair = store.preBulkLoadHFile(finalPath, seqId);
lst.add(pair);
} catch (IOException ioe) {
// A failure here can cause an atomicity violation that we currently
@ -6981,7 +6990,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
" load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
if (bulkLoadListener != null) {
try {
bulkLoadListener.failedBulkLoad(familyName, path);
bulkLoadListener.failedBulkLoad(familyName, finalPath);
} catch (Exception ex) {
LOG.error("Error while calling failedBulkLoad for family " +
Bytes.toString(familyName) + " with path " + path, ex);

View File

@ -508,6 +508,10 @@ public class HRegionFileSystem {
* @throws IOException
*/
Path commitStoreFile(final Path buildPath, Path dstPath) throws IOException {
// rename is not necessary in case of direct-insert stores
if(buildPath.equals(dstPath)){
return dstPath;
}
// buildPath exists, therefore not doing an exists() check.
if (!rename(buildPath, dstPath)) {
throw new IOException("Failed rename of " + buildPath + " to " + dstPath);

View File

@ -28,6 +28,7 @@ import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Consumer;
import org.apache.commons.lang3.StringUtils;
import org.apache.commons.lang3.mutable.MutableInt;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@ -342,7 +343,8 @@ public class SecureBulkLoadManager {
return user;
}
private static class SecureBulkLoadListener implements BulkLoadListener {
//package-private for test purpose only
static class SecureBulkLoadListener implements BulkLoadListener {
// Target filesystem
private final FileSystem fs;
private final String stagingDir;
@ -350,19 +352,28 @@ public class SecureBulkLoadManager {
// Source filesystem
private FileSystem srcFs = null;
private Map<String, FsPermission> origPermissions = null;
private Map<String, String> origSources = null;
public SecureBulkLoadListener(FileSystem fs, String stagingDir, Configuration conf) {
this.fs = fs;
this.stagingDir = stagingDir;
this.conf = conf;
this.origPermissions = new HashMap<>();
this.origSources = new HashMap<>();
}
@Override
public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile)
throws IOException {
public String prepareBulkLoad(final byte[] family, final String srcPath, boolean copyFile,
String customStaging ) throws IOException {
Path p = new Path(srcPath);
Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
//store customStaging for failedBulkLoad
String currentStaging = stagingDir;
if(StringUtils.isNotEmpty(customStaging)){
currentStaging = customStaging;
}
Path stageP = new Path(currentStaging, new Path(Bytes.toString(family), p.getName()));
// In case of Replication for bulk load files, hfiles are already copied in staging directory
if (p.equals(stageP)) {
@ -391,11 +402,16 @@ public class SecureBulkLoadManager {
LOG.debug("Moving " + p + " to " + stageP);
FileStatus origFileStatus = fs.getFileStatus(p);
origPermissions.put(srcPath, origFileStatus.getPermission());
origSources.put(stageP.toString(), srcPath);
if(!fs.rename(p, stageP)) {
throw new IOException("Failed to move HFile: " + p + " to " + stageP);
}
}
fs.setPermission(stageP, PERM_ALL_ACCESS);
if(StringUtils.isNotEmpty(customStaging)) {
fs.setPermission(stageP, PERM_ALL_ACCESS);
}
return stageP.toString();
}
@ -413,35 +429,37 @@ public class SecureBulkLoadManager {
}
@Override
public void failedBulkLoad(final byte[] family, final String srcPath) throws IOException {
public void failedBulkLoad(final byte[] family, final String stagedPath) throws IOException {
try {
Path p = new Path(srcPath);
if (srcFs == null) {
srcFs = FileSystem.newInstance(p.toUri(), conf);
}
if (!FSUtils.isSameHdfs(conf, srcFs, fs)) {
// files are copied so no need to move them back
return;
}
Path stageP = new Path(stagingDir, new Path(Bytes.toString(family), p.getName()));
// In case of Replication for bulk load files, hfiles are not renamed by end point during
// prepare stage, so no need of rename here again
if (p.equals(stageP)) {
LOG.debug(p.getName() + " is already available in source directory. Skipping rename.");
String src = origSources.get(stagedPath);
if(StringUtils.isEmpty(src)){
LOG.debug(stagedPath + " was not moved to staging. No need to move back");
return;
}
LOG.debug("Moving " + stageP + " back to " + p);
if (!fs.rename(stageP, p)) {
throw new IOException("Failed to move HFile: " + stageP + " to " + p);
Path stageP = new Path(stagedPath);
if (!fs.exists(stageP)) {
throw new IOException(
"Missing HFile: " + stageP + ", can't be moved back to it's original place");
}
//we should not move back files if the original exists
Path srcPath = new Path(src);
if(srcFs.exists(srcPath)) {
LOG.debug(src + " is already at it's original place. No need to move.");
return;
}
LOG.debug("Moving " + stageP + " back to " + srcPath);
if (!fs.rename(stageP, srcPath)) {
throw new IOException("Failed to move HFile: " + stageP + " to " + srcPath);
}
// restore original permission
if (origPermissions.containsKey(srcPath)) {
fs.setPermission(p, origPermissions.get(srcPath));
if (origPermissions.containsKey(stagedPath)) {
fs.setPermission(srcPath, origPermissions.get(src));
} else {
LOG.warn("Can't find previous permission for path=" + srcPath);
LOG.warn("Can't find previous permission for path=" + stagedPath);
}
} finally {
closeSrcFs();

View File

@ -54,6 +54,10 @@ public class TestBulkLoad extends TestBulkloadBase {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestBulkLoad.class);
public TestBulkLoad(boolean useFileBasedSFT) {
super(useFileBasedSFT);
}
@Test
public void verifyBulkLoadEvent() throws IOException {
TableName tableName = TableName.valueOf("test", "test");

View File

@ -27,8 +27,10 @@ import java.io.FileOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Random;
import java.util.UUID;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -46,6 +48,7 @@ import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.wal.WAL;
@ -62,7 +65,10 @@ import org.junit.rules.TestName;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
@RunWith(Parameterized.class)
public class TestBulkloadBase {
@ClassRule
public static TemporaryFolder testFolder = new TemporaryFolder();
@ -75,12 +81,31 @@ public class TestBulkloadBase {
protected final byte[] family2 = Bytes.toBytes("family2");
protected final byte[] family3 = Bytes.toBytes("family3");
protected Boolean useFileBasedSFT;
@Rule
public TestName name = new TestName();
public TestBulkloadBase(boolean useFileBasedSFT) {
this.useFileBasedSFT = useFileBasedSFT;
}
@Parameterized.Parameters
public static Collection<Boolean> data() {
Boolean[] data = {false, true};
return Arrays.asList(data);
}
@Before
public void before() throws IOException {
random.nextBytes(randomBytes);
if(useFileBasedSFT) {
conf.set(StoreFileTrackerFactory.TRACKER_IMPL,
"org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
}
else {
conf.unset(StoreFileTrackerFactory.TRACKER_IMPL);
}
}
protected Pair<byte[], String> withMissingHFileForFamily(byte[] family) {
@ -115,7 +140,7 @@ public class TestBulkloadBase {
}
protected HRegion testRegionWithFamilies(byte[]... families) throws IOException {
TableName tableName = TableName.valueOf(name.getMethodName());
TableName tableName = TableName.valueOf(name.getMethodName().substring(0, name.getMethodName().indexOf("[")));
return testRegionWithFamiliesAndSpecifiedTableName(tableName, families);
}
@ -134,7 +159,7 @@ public class TestBulkloadBase {
private String createHFileForFamilies(byte[] family) throws IOException {
HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
// TODO We need a way to do this without creating files
File hFileLocation = testFolder.newFile();
File hFileLocation = testFolder.newFile(generateUniqueName(null));
FSDataOutputStream out = new FSDataOutputStream(new FileOutputStream(hFileLocation), null);
try {
hFileFactory.withOutputStream(out);
@ -153,6 +178,12 @@ public class TestBulkloadBase {
return hFileLocation.getAbsoluteFile().getAbsolutePath();
}
private static String generateUniqueName(final String suffix) {
String name = UUID.randomUUID().toString().replaceAll("-", "");
if (suffix != null) name += suffix;
return name;
}
protected static Matcher<WALEdit> bulkLogWalEditType(byte[] typeBytes) {
return new WalMatcher(typeBytes);
}

View File

@ -61,6 +61,10 @@ public class TestCompactionAfterBulkLoad extends TestBulkloadBase {
private final RegionServerServices regionServerServices = mock(RegionServerServices.class);
public static AtomicInteger called = new AtomicInteger(0);
public TestCompactionAfterBulkLoad(boolean useFileBasedSFT) {
super(useFileBasedSFT);
}
@Override
protected HRegion testRegionWithFamiliesAndSpecifiedTableName(TableName tableName,
byte[]... families) throws IOException {

View File

@ -19,6 +19,8 @@ package org.apache.hadoop.hbase.regionserver;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Collection;
import java.util.Deque;
import java.util.Map;
import java.util.concurrent.ExecutorService;
@ -46,23 +48,29 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContext;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.regionserver.storefiletracker.StoreFileTrackerFactory;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.tool.LoadIncrementalHFiles;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.collect.Multimap;
@RunWith(Parameterized.class)
@Category({RegionServerTests.class, MediumTests.class})
public class TestSecureBulkLoadManager {
@ -87,15 +95,34 @@ public class TestSecureBulkLoadManager {
private Thread laterBulkload;
protected final static HBaseTestingUtility testUtil = new HBaseTestingUtility();
protected Boolean useFileBasedSFT;
private static Configuration conf = testUtil.getConfiguration();
@BeforeClass
public static void setUp() throws Exception {
public TestSecureBulkLoadManager(Boolean useFileBasedSFT) {
this.useFileBasedSFT = useFileBasedSFT;
}
@Parameterized.Parameters
public static Collection<Boolean> data() {
Boolean[] data = {false, true};
return Arrays.asList(data);
}
@Before
public void setUp() throws Exception {
if (useFileBasedSFT) {
conf.set(StoreFileTrackerFactory.TRACKER_IMPL,
"org.apache.hadoop.hbase.regionserver.storefiletracker.FileBasedStoreFileTracker");
}
else{
conf.unset(StoreFileTrackerFactory.TRACKER_IMPL);
}
testUtil.startMiniCluster();
}
@AfterClass
public static void tearDown() throws Exception {
@After
public void tearDown() throws Exception {
testUtil.shutdownMiniCluster();
testUtil.cleanupTestDir();
}

View File

@ -0,0 +1,283 @@
/**
* 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.hadoop.hbase.regionserver;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.BlockLocation;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.rules.TemporaryFolder;
import org.junit.rules.TestName;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.BindException;
import java.net.ServerSocket;
import java.util.Random;
import java.util.UUID;
/**
* Tests for failedBulkLoad logic to make sure staged files are returned to their original location
* if the bulkload have failed.
*/
@Category({MiscTests.class, LargeTests.class})
public class TestSecureBulkloadListener {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestSecureBulkloadListener.class);
private static final Logger LOG = LoggerFactory.getLogger(TestSecureBulkloadListener.class);
@ClassRule
public static TemporaryFolder testFolder = new TemporaryFolder();
private Configuration conf;
private MiniDFSCluster cluster;
private HBaseTestingUtility htu;
private DistributedFileSystem dfs;
private final Random random = new Random();
private final byte[] randomBytes = new byte[100];
private static final String host1 = "host1";
private static final String host2 = "host2";
private static final String host3 = "host3";
private static byte[] FAMILY = Bytes.toBytes("family");
private static final String STAGING_DIR = "staging";
private static final String CUSTOM_STAGING_DIR = "customStaging";
@Rule
public TestName name = new TestName();
@Before
public void setUp() throws Exception {
random.nextBytes(randomBytes);
htu = new HBaseTestingUtility();
htu.getConfiguration().setInt("dfs.blocksize", 1024);// For the test with multiple blocks
htu.getConfiguration().setInt("dfs.replication", 3);
htu.startMiniDFSCluster(3,
new String[]{"/r1", "/r2", "/r3"}, new String[]{host1, host2, host3});
conf = htu.getConfiguration();
cluster = htu.getDFSCluster();
dfs = (DistributedFileSystem) FileSystem.get(conf);
}
@After
public void tearDownAfterClass() throws Exception {
htu.shutdownMiniCluster();
}
@Test
public void testMovingStagedFile() throws Exception {
Path stagingDirPath =
new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR));
if (!dfs.exists(stagingDirPath)) {
dfs.mkdirs(stagingDirPath);
}
SecureBulkLoadManager.SecureBulkLoadListener listener =
new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf);
//creating file to load
String srcFile = createHFileForFamilies(FAMILY);
Path srcPath = new Path(srcFile);
Assert.assertTrue(dfs.exists(srcPath));
Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY)));
if (!dfs.exists(stagedFamily)) {
dfs.mkdirs(stagedFamily);
}
//moving file to staging
String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, false, null);
Path stagedPath = new Path(stagedFile);
Assert.assertTrue(dfs.exists(stagedPath));
Assert.assertFalse(dfs.exists(srcPath));
//moving files back to original location after a failed bulkload
listener.failedBulkLoad(FAMILY, stagedFile);
Assert.assertFalse(dfs.exists(stagedPath));
Assert.assertTrue(dfs.exists(srcPath));
}
@Test
public void testMovingStagedFileWithCustomStageDir() throws Exception {
Path stagingDirPath =
new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR));
if (!dfs.exists(stagingDirPath)) {
dfs.mkdirs(stagingDirPath);
}
SecureBulkLoadManager.SecureBulkLoadListener listener =
new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf);
//creating file to load
String srcFile = createHFileForFamilies(FAMILY);
Path srcPath = new Path(srcFile);
Assert.assertTrue(dfs.exists(srcPath));
Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY)));
if (!dfs.exists(stagedFamily)) {
dfs.mkdirs(stagedFamily);
}
Path customStagingDirPath =
new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), CUSTOM_STAGING_DIR));
Path customStagedFamily = new Path(customStagingDirPath, new Path(Bytes.toString(FAMILY)));
if (!dfs.exists(customStagedFamily)) {
dfs.mkdirs(customStagedFamily);
}
//moving file to staging using a custom staging dir
String stagedFile =
listener.prepareBulkLoad(FAMILY, srcFile, false, customStagingDirPath.toString());
Path stagedPath = new Path(stagedFile);
Assert.assertTrue(dfs.exists(stagedPath));
Assert.assertFalse(dfs.exists(srcPath));
//moving files back to original location after a failed bulkload
listener.failedBulkLoad(FAMILY, stagedFile);
Assert.assertFalse(dfs.exists(stagedPath));
Assert.assertTrue(dfs.exists(srcPath));
}
@Test
public void testCopiedStagedFile() throws Exception {
Path stagingDirPath =
new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR));
if (!dfs.exists(stagingDirPath)) {
dfs.mkdirs(stagingDirPath);
}
SecureBulkLoadManager.SecureBulkLoadListener listener =
new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf);
//creating file to load
String srcFile = createHFileForFamilies(FAMILY);
Path srcPath = new Path(srcFile);
Assert.assertTrue(dfs.exists(srcPath));
Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY)));
if (!dfs.exists(stagedFamily)) {
dfs.mkdirs(stagedFamily);
}
//copying file to staging
String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, true, null);
Path stagedPath = new Path(stagedFile);
Assert.assertTrue(dfs.exists(stagedPath));
Assert.assertTrue(dfs.exists(srcPath));
//should do nothing because the original file was copied to staging
listener.failedBulkLoad(FAMILY, stagedFile);
Assert.assertTrue(dfs.exists(stagedPath));
Assert.assertTrue(dfs.exists(srcPath));
}
@Test(expected = IOException.class)
public void testDeletedStagedFile() throws Exception {
Path stagingDirPath =
new Path(dfs.getWorkingDirectory(), new Path(name.getMethodName(), STAGING_DIR));
if (!dfs.exists(stagingDirPath)) {
dfs.mkdirs(stagingDirPath);
}
SecureBulkLoadManager.SecureBulkLoadListener listener =
new SecureBulkLoadManager.SecureBulkLoadListener(dfs, stagingDirPath.toString(), conf);
//creating file to load
String srcFile = createHFileForFamilies(FAMILY);
Path srcPath = new Path(srcFile);
Assert.assertTrue(dfs.exists(srcPath));
Path stagedFamily = new Path(stagingDirPath, new Path(Bytes.toString(FAMILY)));
if (!dfs.exists(stagedFamily)) {
dfs.mkdirs(stagedFamily);
}
//moving file to staging
String stagedFile = listener.prepareBulkLoad(FAMILY, srcFile, false, null);
Path stagedPath = new Path(stagedFile);
Assert.assertTrue(dfs.exists(stagedPath));
Assert.assertFalse(dfs.exists(srcPath));
dfs.delete(stagedPath, false);
//moving files back to original location after a failed bulkload
listener.failedBulkLoad(FAMILY, stagedFile);
}
private String createHFileForFamilies(byte[] family) throws IOException {
HFile.WriterFactory hFileFactory = HFile.getWriterFactoryNoCache(conf);
Path testDir = new Path(dfs.getWorkingDirectory() , new Path(name.getMethodName(), Bytes.toString(family)));
if(!dfs.exists(testDir)){
dfs.mkdirs(testDir);
}
Path hfilePath = new Path(testDir, generateUniqueName(null));
FSDataOutputStream out = dfs.createFile(hfilePath).build();
try {
hFileFactory.withOutputStream(out);
hFileFactory.withFileContext(new HFileContextBuilder().build());
HFile.Writer writer = hFileFactory.create();
try {
writer.append(new KeyValue(ExtendedCellBuilderFactory.create(CellBuilderType.DEEP_COPY)
.setRow(randomBytes).setFamily(family).setQualifier(randomBytes).setTimestamp(0L)
.setType(KeyValue.Type.Put.getCode()).setValue(randomBytes).build()));
} finally {
writer.close();
}
} finally {
out.close();
}
return hfilePath.toString();
}
private static String generateUniqueName(final String suffix) {
String name = UUID.randomUUID().toString().replaceAll("-", "");
if (suffix != null) name += suffix;
return name;
}
}