HBASE-14464 Removed unused fs code

This commit is contained in:
Matteo Bertozzi 2015-09-23 09:40:09 -07:00
parent 2c83d8a263
commit 5d9ffffc07
20 changed files with 12 additions and 730 deletions

View File

@ -41,7 +41,6 @@ import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableDescriptor;
@ -558,43 +557,6 @@ public class MasterFileSystem {
}
}
public void deleteRegion(HRegionInfo region) throws IOException {
HFileArchiver.archiveRegion(conf, fs, region);
}
public void deleteTable(TableName tableName) throws IOException {
fs.delete(FSUtils.getTableDir(rootdir, tableName), true);
}
/**
* Move the specified table to the hbase temp directory
* @param tableName Table name to move
* @return The temp location of the table moved
* @throws IOException in case of file-system failure
*/
public Path moveTableToTemp(TableName tableName) throws IOException {
Path srcPath = FSUtils.getTableDir(rootdir, tableName);
Path tempPath = FSUtils.getTableDir(this.tempdir, tableName);
// Ensure temp exists
if (!fs.exists(tempPath.getParent()) && !fs.mkdirs(tempPath.getParent())) {
throw new IOException("HBase temp directory '" + tempPath.getParent() + "' creation failure.");
}
if (!fs.rename(srcPath, tempPath)) {
throw new IOException("Unable to move '" + srcPath + "' to temp '" + tempPath + "'");
}
return tempPath;
}
public void updateRegionInfo(HRegionInfo region) {
// TODO implement this. i think this is currently broken in trunk i don't
// see this getting updated.
// @see HRegion.checkRegioninfoOnFilesystem()
}
public void deleteFamilyFromFS(HRegionInfo region, byte[] familyName, boolean hasMob)
throws IOException {
// archive family store files
@ -638,67 +600,6 @@ public class MasterFileSystem {
}
}
/**
* Delete column of a table
* @param tableName
* @param familyName
* @return Modified HTableDescriptor with requested column deleted.
* @throws IOException
*/
public HTableDescriptor deleteColumn(TableName tableName, byte[] familyName)
throws IOException {
LOG.info("DeleteColumn. Table = " + tableName
+ " family = " + Bytes.toString(familyName));
HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
htd.removeFamily(familyName);
this.services.getTableDescriptors().add(htd);
return htd;
}
/**
* Modify Column of a table
* @param tableName
* @param hcd HColumnDesciptor
* @return Modified HTableDescriptor with the column modified.
* @throws IOException
*/
public HTableDescriptor modifyColumn(TableName tableName, HColumnDescriptor hcd)
throws IOException {
LOG.info("AddModifyColumn. Table = " + tableName
+ " HCD = " + hcd.toString());
HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
byte [] familyName = hcd.getName();
if(!htd.hasFamily(familyName)) {
throw new InvalidFamilyOperationException("Family '" +
Bytes.toString(familyName) + "' doesn't exists so cannot be modified");
}
htd.modifyFamily(hcd);
this.services.getTableDescriptors().add(htd);
return htd;
}
/**
* Add column to a table
* @param tableName
* @param hcd
* @return Modified HTableDescriptor with new column added.
* @throws IOException
*/
public HTableDescriptor addColumn(TableName tableName, HColumnDescriptor hcd)
throws IOException {
LOG.info("AddColumn. Table = " + tableName + " HCD = " +
hcd.toString());
HTableDescriptor htd = this.services.getTableDescriptors().get(tableName);
if (htd == null) {
throw new InvalidFamilyOperationException("Family '" +
hcd.getNameAsString() + "' cannot be modified as HTD is null");
}
htd.addFamily(hcd);
this.services.getTableDescriptors().add(htd);
return htd;
}
/**
* The function is used in SSH to set recovery mode based on configuration after all outstanding
* log split tasks drained.

View File

@ -1,103 +0,0 @@
/**
* 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.master.snapshot;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.master.cleaner.BaseLogCleanerDelegate;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.FSUtils;
/**
* Implementation of a log cleaner that checks if a log is still used by
* snapshots of HBase tables.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
@InterfaceStability.Evolving
public class SnapshotLogCleaner extends BaseLogCleanerDelegate {
private static final Log LOG = LogFactory.getLog(SnapshotLogCleaner.class);
/**
* Conf key for the frequency to attempt to refresh the cache of hfiles currently used in
* snapshots (ms)
*/
static final String WAL_CACHE_REFRESH_PERIOD_CONF_KEY =
"hbase.master.hlogcleaner.plugins.snapshot.period";
/** Refresh cache, by default, every 5 minutes */
private static final long DEFAULT_WAL_CACHE_REFRESH_PERIOD = 300000;
private SnapshotFileCache cache;
@Override
public synchronized Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
try {
return cache.getUnreferencedFiles(files);
} catch (IOException e) {
LOG.error("Exception while checking if files were valid, keeping them just in case.", e);
return Collections.emptyList();
}
}
/**
* This method should only be called <b>once</b>, as it starts a thread to keep the cache
* up-to-date.
* <p>
* {@inheritDoc}
*/
@Override
public void setConf(Configuration conf) {
super.setConf(conf);
try {
long cacheRefreshPeriod = conf.getLong(
WAL_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_WAL_CACHE_REFRESH_PERIOD);
final FileSystem fs = FSUtils.getCurrentFileSystem(conf);
Path rootDir = FSUtils.getRootDir(conf);
cache = new SnapshotFileCache(fs, rootDir, cacheRefreshPeriod, cacheRefreshPeriod,
"snapshot-log-cleaner-cache-refresher", new SnapshotFileCache.SnapshotFileInspector() {
public Collection<String> filesUnderSnapshot(final Path snapshotDir)
throws IOException {
return SnapshotReferenceUtil.getWALNames(fs, snapshotDir);
}
});
} catch (IOException e) {
LOG.error("Failed to create snapshot log cleaner", e);
}
}
@Override
public void stop(String why) {
this.cache.stop(why);
}
@Override
public boolean isStopped() {
return this.cache.isStopped();
}
}

View File

@ -1028,7 +1028,6 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
// Inject snapshot cleaners, if snapshot.enable is true
hfileCleaners.add(SnapshotHFileCleaner.class.getName());
hfileCleaners.add(HFileLinkCleaner.class.getName());
logCleaners.add(SnapshotLogCleaner.class.getName());
// Set cleaners conf
conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
@ -1037,7 +1036,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable
logCleaners.toArray(new String[logCleaners.size()]));
} else {
// Verify if cleaners are present
snapshotEnabled = logCleaners.contains(SnapshotLogCleaner.class.getName()) &&
snapshotEnabled =
hfileCleaners.contains(SnapshotHFileCleaner.class.getName()) &&
hfileCleaners.contains(HFileLinkCleaner.class.getName());

View File

@ -209,8 +209,7 @@ public class ExportSnapshot extends Configured implements Tool {
new Path(region, new Path(family, hfile)));
break;
case WAL:
Path oldLogsDir = new Path(outputRoot, HConstants.HREGION_OLDLOGDIR_NAME);
path = new Path(oldLogsDir, inputInfo.getWalName());
LOG.warn("snapshot does not keeps WALs: " + inputInfo);
break;
default:
throw new IOException("Invalid File Type: " + inputInfo.getType().toString());
@ -559,19 +558,6 @@ public class ExportSnapshot extends Configured implements Tool {
files.add(new Pair<SnapshotFileInfo, Long>(fileInfo, size));
}
}
@Override
public void logFile (final String server, final String logfile)
throws IOException {
SnapshotFileInfo fileInfo = SnapshotFileInfo.newBuilder()
.setType(SnapshotFileInfo.Type.WAL)
.setWalServer(server)
.setWalName(logfile)
.build();
long size = new WALLink(conf, server, logfile).getFileStatus(fs).getLen();
files.add(new Pair<SnapshotFileInfo, Long>(fileInfo, size));
}
});
return files;

View File

@ -39,8 +39,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;

View File

@ -430,20 +430,6 @@ public final class SnapshotInfo extends Configured implements Tool {
state == null ? "" : "(" + state + ")");
}
}
@Override
public void logFile (final String server, final String logfile)
throws IOException {
SnapshotStats.FileInfo info = stats.addLogFile(server, logfile);
if (showFiles) {
String state = info.getStateToString();
System.out.printf("%8s log %s on server %s (%s)%n",
(info.isMissing() ? "-" : fileSizeToString(info.getSize())),
logfile, server,
state == null ? "" : "(" + state + ")");
}
}
});
// Dump the stats
@ -518,11 +504,6 @@ public final class SnapshotInfo extends Configured implements Tool {
stats.addStoreFile(regionInfo, family, storeFile);
}
}
@Override
public void logFile (final String server, final String logfile) throws IOException {
stats.addLogFile(server, logfile);
}
});
return stats;
}

View File

@ -65,7 +65,7 @@ import org.apache.hadoop.hbase.util.Threads;
* On open() the snapshot will be loaded to the current in-memory format.
*/
@InterfaceAudience.Private
public class SnapshotManifest {
public final class SnapshotManifest {
private static final Log LOG = LogFactory.getLog(SnapshotManifest.class);
private static final String DATA_MANIFEST_NAME = "data.manifest";

View File

@ -54,7 +54,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
* /snapshotName/regionName/familyName/fileName
*/
@InterfaceAudience.Private
public class SnapshotManifestV1 {
public final class SnapshotManifestV1 {
private static final Log LOG = LogFactory.getLog(SnapshotManifestV1.class);
public static final int DESCRIPTOR_VERSION = 0;
@ -154,7 +154,7 @@ public class SnapshotManifestV1 {
fs.delete(new Path(snapshotDir, regionName), true);
}
static SnapshotRegionManifest buildManifestFromDisk (final Configuration conf,
static SnapshotRegionManifest buildManifestFromDisk(final Configuration conf,
final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
HRegionFileSystem regionFs = HRegionFileSystem.openRegionFromFileSystem(conf, fs,
tableDir, regionInfo, true);

View File

@ -53,13 +53,15 @@ import org.apache.hadoop.hbase.util.FSUtils;
* /snapshotName/manifest.regionName
*/
@InterfaceAudience.Private
public class SnapshotManifestV2 {
public final class SnapshotManifestV2 {
private static final Log LOG = LogFactory.getLog(SnapshotManifestV2.class);
public static final int DESCRIPTOR_VERSION = 2;
private static final String SNAPSHOT_MANIFEST_PREFIX = "region-manifest.";
private SnapshotManifestV2() {}
static class ManifestBuilder implements SnapshotManifest.RegionVisitor<
SnapshotRegionManifest.Builder, SnapshotRegionManifest.FamilyFiles.Builder> {
private final Configuration conf;

View File

@ -42,9 +42,7 @@ import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.wal.DefaultWALProvider;
import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
import org.apache.hadoop.hbase.util.FSVisitor;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
/**
@ -59,8 +57,7 @@ public final class SnapshotReferenceUtil {
final SnapshotRegionManifest.StoreFile storeFile) throws IOException;
}
public interface SnapshotVisitor extends StoreFileVisitor,
FSVisitor.LogFileVisitor {
public interface SnapshotVisitor extends StoreFileVisitor {
}
private SnapshotReferenceUtil() {
@ -68,18 +65,7 @@ public final class SnapshotReferenceUtil {
}
/**
* Get log directory for a server in a snapshot.
*
* @param snapshotDir directory where the specific snapshot is stored
* @param serverName name of the parent regionserver for the log files
* @return path to the log home directory for the archive files.
*/
public static Path getLogsDir(Path snapshotDir, String serverName) {
return new Path(snapshotDir, DefaultWALProvider.getWALDirectoryName(serverName));
}
/**
* Iterate over the snapshot store files, restored.edits and logs
* Iterate over the snapshot store files
*
* @param conf The current {@link Configuration} instance.
* @param fs {@link FileSystem}
@ -108,7 +94,6 @@ public final class SnapshotReferenceUtil {
final Path snapshotDir, final SnapshotDescription desc, final SnapshotVisitor visitor)
throws IOException {
visitTableStoreFiles(conf, fs, snapshotDir, desc, visitor);
visitLogFiles(fs, snapshotDir, visitor);
}
/**©
@ -154,19 +139,6 @@ public final class SnapshotReferenceUtil {
}
}
/**
* Iterate over the snapshot log files
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @param visitor callback object to get the log files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitLogFiles(final FileSystem fs, final Path snapshotDir,
final FSVisitor.LogFileVisitor visitor) throws IOException {
FSVisitor.visitLogFiles(fs, snapshotDir, visitor);
}
/**
* Verify the validity of the snapshot
*
@ -367,24 +339,4 @@ public final class SnapshotReferenceUtil {
});
return names;
}
/**
* Returns the log file names available in the snapshot.
*
* @param fs {@link FileSystem}
* @param snapshotDir {@link Path} to the Snapshot directory
* @throws IOException if an error occurred while scanning the directory
* @return the names of wals in the specified snaphot
*/
public static Set<String> getWALNames(final FileSystem fs, final Path snapshotDir)
throws IOException {
final Set<String> names = new HashSet<String>();
visitLogFiles(fs, snapshotDir, new FSVisitor.LogFileVisitor() {
@Override
public void logFile (final String server, final String logfile) throws IOException {
names.add(logfile);
}
});
return names;
}
}

View File

@ -1053,44 +1053,6 @@ public abstract class FSUtils {
return blocksDistribution;
}
/**
* Runs through the hbase rootdir and checks all stores have only
* one file in them -- that is, they've been major compacted. Looks
* at root and meta tables too.
* @param fs filesystem
* @param hbaseRootDir hbase root directory
* @return True if this hbase install is major compacted.
* @throws IOException e
*/
public static boolean isMajorCompacted(final FileSystem fs,
final Path hbaseRootDir)
throws IOException {
List<Path> tableDirs = getTableDirs(fs, hbaseRootDir);
PathFilter regionFilter = new RegionDirFilter(fs);
PathFilter familyFilter = new FamilyDirFilter(fs);
for (Path d : tableDirs) {
FileStatus[] regionDirs = fs.listStatus(d, regionFilter);
for (FileStatus regionDir : regionDirs) {
Path dd = regionDir.getPath();
// Else its a region name. Now look in region for families.
FileStatus[] familyDirs = fs.listStatus(dd, familyFilter);
for (FileStatus familyDir : familyDirs) {
Path family = familyDir.getPath();
// Now in family make sure only one file.
FileStatus[] familyStatus = fs.listStatus(family);
if (familyStatus.length > 1) {
LOG.debug(family.toString() + " has " + familyStatus.length +
" files.");
return false;
}
}
}
}
return true;
}
// TODO move this method OUT of FSUtils. No dependencies to HMaster
/**
* Returns the total overall fragmentation percentage. Includes hbase:meta and
@ -2057,7 +2019,7 @@ public abstract class FSUtils {
/**
* @param c
* @return The DFSClient DFSHedgedReadMetrics instance or null if can't be found or not on hdfs.
* @throws IOException
* @throws IOException
*/
public static DFSHedgedReadMetrics getDFSHedgedReadMetrics(final Configuration c)
throws IOException {

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hbase.util;
import java.io.IOException;
import java.util.NavigableSet;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -28,8 +27,6 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.wal.WALSplitter;
/**
* Utility methods for interacting with the hbase.root file system.
@ -38,52 +35,15 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
public final class FSVisitor {
private static final Log LOG = LogFactory.getLog(FSVisitor.class);
public interface RegionVisitor {
void region(final String region) throws IOException;
}
public interface StoreFileVisitor {
void storeFile(final String region, final String family, final String hfileName)
throws IOException;
}
public interface RecoveredEditsVisitor {
void recoveredEdits (final String region, final String logfile)
throws IOException;
}
public interface LogFileVisitor {
void logFile (final String server, final String logfile)
throws IOException;
}
private FSVisitor() {
// private constructor for utility class
}
/**
* Iterate over the table store files
*
* @param fs {@link FileSystem}
* @param tableDir {@link Path} to the table directory
* @param visitor callback object to get the store files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRegions(final FileSystem fs, final Path tableDir,
final RegionVisitor visitor) throws IOException {
FileStatus[] regions = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regions == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No regions under directory:" + tableDir);
}
return;
}
for (FileStatus region: regions) {
visitor.region(region.getPath().getName());
}
}
/**
* Iterate over the table store files
*
@ -145,85 +105,4 @@ public final class FSVisitor {
}
}
}
/**
* Iterate over each region in the table and inform about recovered.edits
*
* @param fs {@link FileSystem}
* @param tableDir {@link Path} to the table directory
* @param visitor callback object to get the recovered.edits files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitTableRecoveredEdits(final FileSystem fs, final Path tableDir,
final FSVisitor.RecoveredEditsVisitor visitor) throws IOException {
FileStatus[] regions = FSUtils.listStatus(fs, tableDir, new FSUtils.RegionDirFilter(fs));
if (regions == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No recoveredEdits regions under directory:" + tableDir);
}
return;
}
for (FileStatus region: regions) {
visitRegionRecoveredEdits(fs, region.getPath(), visitor);
}
}
/**
* Iterate over recovered.edits of the specified region
*
* @param fs {@link FileSystem}
* @param regionDir {@link Path} to the Region directory
* @param visitor callback object to get the recovered.edits files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitRegionRecoveredEdits(final FileSystem fs, final Path regionDir,
final FSVisitor.RecoveredEditsVisitor visitor) throws IOException {
NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(fs, regionDir);
if (files == null || files.size() == 0) return;
for (Path source: files) {
// check to see if the file is zero length, in which case we can skip it
FileStatus stat = fs.getFileStatus(source);
if (stat.getLen() <= 0) continue;
visitor.recoveredEdits(regionDir.getName(), source.getName());
}
}
/**
* Iterate over hbase log files
*
* @param fs {@link FileSystem}
* @param rootDir {@link Path} to the HBase root folder
* @param visitor callback object to get the log files
* @throws IOException if an error occurred while scanning the directory
*/
public static void visitLogFiles(final FileSystem fs, final Path rootDir,
final LogFileVisitor visitor) throws IOException {
Path logsDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
FileStatus[] logServerDirs = FSUtils.listStatus(fs, logsDir);
if (logServerDirs == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No logs under directory:" + logsDir);
}
return;
}
for (FileStatus serverLogs: logServerDirs) {
String serverName = serverLogs.getPath().getName();
FileStatus[] wals = FSUtils.listStatus(fs, serverLogs.getPath());
if (wals == null) {
if (LOG.isTraceEnabled()) {
LOG.trace("No wals found for server: " + serverName + ", skipping.");
}
continue;
}
for (FileStatus walRef: wals) {
visitor.logFile(serverName, walRef.getPath().getName());
}
}
}
}

View File

@ -70,7 +70,6 @@ public class TestNamespace {
@BeforeClass
public static void setUp() throws Exception {
TEST_UTIL = new HBaseTestingUtility();
TEST_UTIL.getConfiguration().setInt("hbase.namespacejanitor.interval", 5000);
TEST_UTIL.startMiniCluster(NUM_SLAVES_BASE);
admin = TEST_UTIL.getHBaseAdmin();
cluster = TEST_UTIL.getHBaseCluster();
@ -290,33 +289,6 @@ public class TestNamespace {
admin.deleteTable(desc.getTableName());
}
@Ignore @Test
public void testNamespaceJanitor() throws Exception {
FileSystem fs = TEST_UTIL.getTestFileSystem();
int fsCount = fs.listStatus(new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
HConstants.BASE_NAMESPACE_DIR)).length;
Path fakeNSPath =
FSUtils.getNamespaceDir(FSUtils.getRootDir(TEST_UTIL.getConfiguration()), "foo");
assertTrue(fs.mkdirs(fakeNSPath));
String fakeZnode = ZKUtil.joinZNode(ZooKeeperWatcher.namespaceZNode, "foo");
int zkCount = ZKUtil.listChildrenNoWatch(TEST_UTIL.getZooKeeperWatcher(),
ZooKeeperWatcher.namespaceZNode).size();
ZKUtil.createWithParents(TEST_UTIL.getZooKeeperWatcher(), fakeZnode);
Thread.sleep(10000);
//verify namespace count is the same and orphan is removed
assertFalse(fs.exists(fakeNSPath));
assertEquals(fsCount, fs.listStatus(new Path(FSUtils.getRootDir(TEST_UTIL.getConfiguration()),
HConstants.BASE_NAMESPACE_DIR)).length);
assertEquals(-1, ZKUtil.checkExists(TEST_UTIL.getZooKeeperWatcher(), fakeZnode));
assertEquals(zkCount,
ZKUtil.listChildrenNoWatch(TEST_UTIL.getZooKeeperWatcher(),
ZooKeeperWatcher.namespaceZNode).size());
}
@Test(timeout = 60000)
public void testNamespaceOperations() throws IOException {
admin.createNamespace(NamespaceDescriptor.create(prefix + "ns1").build());

View File

@ -1254,11 +1254,6 @@ public class TestMasterObserver {
Configuration conf = UTIL.getConfiguration();
conf.set(CoprocessorHost.MASTER_COPROCESSOR_CONF_KEY,
CPMasterObserver.class.getName());
conf.set("hbase.master.hfilecleaner.plugins",
"org.apache.hadoop.hbase.master.cleaner.HFileLinkCleaner," +
"org.apache.hadoop.hbase.master.snapshot.SnapshotHFileCleaner");
conf.set("hbase.master.logcleaner.plugins",
"org.apache.hadoop.hbase.master.snapshot.SnapshotLogCleaner");
// We need more than one data server on this test
UTIL.startMiniCluster(2);
}

View File

@ -95,41 +95,6 @@ public class TestSnapshotFileCache {
createAndTestSnapshotV2(cache, "snapshot2b", true, true);
}
@Test
public void testJustFindLogsDirectory() throws Exception {
// don't refresh the cache unless we tell it to
long period = Long.MAX_VALUE;
Path snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
SnapshotFileCache cache = new SnapshotFileCache(fs, rootDir, period, 10000000,
"test-snapshot-file-cache-refresh", new SnapshotFileCache.SnapshotFileInspector() {
public Collection<String> filesUnderSnapshot(final Path snapshotDir)
throws IOException {
return SnapshotReferenceUtil.getWALNames(fs, snapshotDir);
}
});
// create a file in a 'completed' snapshot
SnapshotDescription desc = SnapshotDescription.newBuilder().setName("snapshot").build();
Path snapshot = SnapshotDescriptionUtils.getCompletedSnapshotDir(desc, rootDir);
SnapshotDescriptionUtils.writeSnapshotInfo(desc, snapshot, fs);
Path file1 = new Path(new Path(new Path(snapshot, "7e91021"), "fam"), "file1");
fs.createNewFile(file1);
// and another file in the logs directory
Path logs = SnapshotReferenceUtil.getLogsDir(snapshot, "server");
Path log = new Path(logs, "me.hbase.com%2C58939%2C1350424310315.1350424315552");
fs.createNewFile(log);
FSUtils.logFileSystemState(fs, rootDir, LOG);
// then make sure the cache only finds the log files
Iterable<FileStatus> notSnapshot = getNonSnapshotFiles(cache, file1);
assertFalse("Cache found '" + file1 + "', but it shouldn't have.",
Iterables.contains(notSnapshot, file1.getName()));
notSnapshot = getNonSnapshotFiles(cache, log);
assertTrue("Cache didn't find:" + log, !Iterables.contains(notSnapshot, log));
}
@Test
public void testReloadModifiedDirectory() throws IOException {
// don't refresh the cache unless we tell it to
@ -215,10 +180,6 @@ public class TestSnapshotFileCache {
SnapshotReferenceUtil
.visitReferencedFiles(UTIL.getConfiguration(), fs, builder.getSnapshotsDir(),
new SnapshotReferenceUtil.SnapshotVisitor() {
@Override public void logFile(String server, String logfile) throws IOException {
// do nothing.
}
@Override public void storeFile(HRegionInfo regionInfo, String familyName,
SnapshotProtos.SnapshotRegionManifest.StoreFile storeFile) throws IOException {
FileStatus status = mockStoreFile(storeFile.getName());
@ -239,7 +200,6 @@ public class TestSnapshotFileCache {
class SnapshotFiles implements SnapshotFileCache.SnapshotFileInspector {
public Collection<String> filesUnderSnapshot(final Path snapshotDir) throws IOException {
Collection<String> files = new HashSet<String>();
files.addAll(SnapshotReferenceUtil.getWALNames(fs, snapshotDir));
files.addAll(SnapshotReferenceUtil.getHFileNames(UTIL.getConfiguration(), fs, snapshotDir));
return files;
}

View File

@ -1,86 +0,0 @@
/**
* 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.master.snapshot;
import static org.junit.Assert.assertFalse;
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.FSUtils;
import org.junit.AfterClass;
import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
* Test that the snapshot log cleaner finds logs referenced in a snapshot
*/
@Category({MasterTests.class, SmallTests.class})
public class TestSnapshotLogCleaner {
private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@AfterClass
public static void cleanup() throws IOException {
Configuration conf = TEST_UTIL.getConfiguration();
Path rootDir = FSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(conf);
// cleanup
fs.delete(rootDir, true);
}
@Test
public void testFindsSnapshotFilesWhenCleaning() throws IOException {
Configuration conf = TEST_UTIL.getConfiguration();
FSUtils.setRootDir(conf, TEST_UTIL.getDataTestDir());
Path rootDir = FSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(conf);
SnapshotLogCleaner cleaner = new SnapshotLogCleaner();
cleaner.setConf(conf);
// write an hfile to the snapshot directory
String snapshotName = "snapshot";
byte[] snapshot = Bytes.toBytes(snapshotName);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, rootDir);
Path snapshotLogDir = new Path(snapshotDir, HConstants.HREGION_LOGDIR_NAME);
String timestamp = "1339643343027";
String hostFromMaster = "localhost%2C59648%2C1339643336601";
Path hostSnapshotLogDir = new Path(snapshotLogDir, hostFromMaster);
String snapshotlogfile = hostFromMaster + "." + timestamp + ".hbase";
// add the reference to log in the snapshot
fs.create(new Path(hostSnapshotLogDir, snapshotlogfile));
// now check to see if that log file would get deleted.
Path oldlogDir = new Path(rootDir, HConstants.HREGION_OLDLOGDIR_NAME);
Path logFile = new Path(oldlogDir, snapshotlogfile);
fs.create(logFile);
// make sure that the file isn't deletable
assertFalse(cleaner.isFileDeletable(fs.getFileStatus(logFile)));
}
}

View File

@ -121,7 +121,6 @@ public class TestSnapshotManager {
conf = new Configuration();
conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
SnapshotHFileCleaner.class.getName(), HFileLinkCleaner.class.getName());
conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, SnapshotLogCleaner.class.getName());
conf.setBoolean(SnapshotManager.HBASE_SNAPSHOT_ENABLED, false);
manager = getNewManager(conf);
assertFalse("Snapshot should be disabled", isSnapshotSupported(manager));
@ -130,7 +129,6 @@ public class TestSnapshotManager {
conf = new Configuration();
conf.setStrings(HFileCleaner.MASTER_HFILE_CLEANER_PLUGINS,
SnapshotHFileCleaner.class.getName(), HFileLinkCleaner.class.getName());
conf.set(HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS, SnapshotLogCleaner.class.getName());
manager = getNewManager(conf);
assertTrue("Snapshot should be enabled, because cleaners are present",
isSnapshotSupported(manager));

View File

@ -381,13 +381,6 @@ public class TestExportSnapshot {
}
}
@Override
public void logFile (final String server, final String logfile)
throws IOException {
snapshotFiles.add(logfile);
verifyNonEmptyFile(new Path(exportedSnapshot, new Path(server, logfile)));
}
private void verifyNonEmptyFile(final Path path) throws IOException {
assertTrue(path + " should exists", fs.exists(path));
assertTrue(path + " should not be empty", fs.getFileStatus(path).getLen() > 0);

View File

@ -381,13 +381,6 @@ public class TestMobExportSnapshot {
}
}
@Override
public void logFile (final String server, final String logfile)
throws IOException {
snapshotFiles.add(logfile);
verifyNonEmptyFile(new Path(exportedSnapshot, new Path(server, logfile)));
}
private void verifyNonEmptyFile(final Path path) throws IOException {
assertTrue(path + " should exists", fs.exists(path));
assertTrue(path + " should not be empty", fs.getFileStatus(path).getLen() > 0);

View File

@ -51,31 +51,21 @@ public class TestFSVisitor {
private Set<String> tableFamilies;
private Set<String> tableRegions;
private Set<String> recoveredEdits;
private Set<String> tableHFiles;
private Set<String> regionServers;
private Set<String> serverLogs;
private FileSystem fs;
private Path tableDir;
private Path logsDir;
private Path rootDir;
@Before
public void setUp() throws Exception {
fs = FileSystem.get(TEST_UTIL.getConfiguration());
rootDir = TEST_UTIL.getDataTestDir("hbase");
logsDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
tableFamilies = new HashSet<String>();
tableRegions = new HashSet<String>();
recoveredEdits = new HashSet<String>();
tableHFiles = new HashSet<String>();
regionServers = new HashSet<String>();
serverLogs = new HashSet<String>();
tableDir = createTableFiles(rootDir, TABLE_NAME, tableRegions, tableFamilies, tableHFiles);
createRecoverEdits(tableDir, tableRegions, recoveredEdits);
createLogs(logsDir, regionServers, serverLogs);
FSUtils.logFileSystemState(fs, rootDir, LOG);
}
@ -102,36 +92,6 @@ public class TestFSVisitor {
assertEquals(tableHFiles, hfiles);
}
@Test
public void testVisitRecoveredEdits() throws IOException {
final Set<String> regions = new HashSet<String>();
final Set<String> edits = new HashSet<String>();
FSVisitor.visitTableRecoveredEdits(fs, tableDir, new FSVisitor.RecoveredEditsVisitor() {
public void recoveredEdits (final String region, final String logfile)
throws IOException {
regions.add(region);
edits.add(logfile);
}
});
assertEquals(tableRegions, regions);
assertEquals(recoveredEdits, edits);
}
@Test
public void testVisitLogFiles() throws IOException {
final Set<String> servers = new HashSet<String>();
final Set<String> logs = new HashSet<String>();
FSVisitor.visitLogFiles(fs, rootDir, new FSVisitor.LogFileVisitor() {
public void logFile (final String server, final String logfile) throws IOException {
servers.add(server);
logs.add(logfile);
}
});
assertEquals(regionServers, servers);
assertEquals(serverLogs, logs);
}
/*
* |-testtb/
* |----f1d3ff8443297732862df21dc4e57262/
@ -163,66 +123,4 @@ public class TestFSVisitor {
}
return tableDir;
}
/*
* |-testtb/
* |----f1d3ff8443297732862df21dc4e57262/
* |-------recovered.edits/
* |----------0000001351969633479
* |----------0000001351969633481
*/
private void createRecoverEdits(final Path tableDir, final Set<String> tableRegions,
final Set<String> recoverEdits) throws IOException {
for (String region: tableRegions) {
Path regionEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(new Path(tableDir, region));
long seqId = System.currentTimeMillis();
for (int i = 0; i < 3; ++i) {
String editName = String.format("%019d", seqId + i);
recoverEdits.add(editName);
FSDataOutputStream stream = fs.create(new Path(regionEditsDir, editName));
stream.write(Bytes.toBytes("test"));
stream.close();
}
}
}
/*
* Old style
* |-.logs/
* |----server5,5,1351969633508/
* |-------server5,5,1351969633508.0
* |----server6,6,1351969633512/
* |-------server6,6,1351969633512.0
* |-------server6,6,1351969633512.3
* New style
* |-.logs/
* |----server3,5,1351969633508/
* |-------server3,5,1351969633508.default.0
* |----server4,6,1351969633512/
* |-------server4,6,1351969633512.default.0
* |-------server4,6,1351969633512.some_provider.3
*/
private void createLogs(final Path logDir, final Set<String> servers,
final Set<String> logs) throws IOException {
for (int s = 0; s < 7; ++s) {
String server = String.format("server%d,%d,%d", s, s, System.currentTimeMillis());
servers.add(server);
Path serverLogDir = new Path(logDir, server);
if (s % 2 == 0) {
if (s % 3 == 0) {
server += ".default";
} else {
server += "." + s;
}
}
fs.mkdirs(serverLogDir);
for (int i = 0; i < 5; ++i) {
String logfile = server + '.' + i;
logs.add(logfile);
FSDataOutputStream stream = fs.create(new Path(serverLogDir, logfile));
stream.write(Bytes.toBytes("test"));
stream.close();
}
}
}
}