HBASE-531 Merge tool won't merge two overlapping regions (port HBASE-483 to trunk) (See HBASE-483 for list of changes)
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@639775 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ef1190ae28
commit
a9d8d51663
@ -47,6 +47,8 @@ Hbase Change Log
|
|||||||
HBASE-525 HTable.getRow(Text) does not work (Clint Morgan via Bryan Duxbury)
|
HBASE-525 HTable.getRow(Text) does not work (Clint Morgan via Bryan Duxbury)
|
||||||
HBASE-524 Problems with getFull
|
HBASE-524 Problems with getFull
|
||||||
HBASE-528 table 'does not exist' when it does
|
HBASE-528 table 'does not exist' when it does
|
||||||
|
HBASE-531 Merge tool won't merge two overlapping regions (port HBASE-483 to
|
||||||
|
trunk)
|
||||||
|
|
||||||
IMPROVEMENTS
|
IMPROVEMENTS
|
||||||
HBASE-415 Rewrite leases to use DelayedBlockingQueue instead of polling
|
HBASE-415 Rewrite leases to use DelayedBlockingQueue instead of polling
|
||||||
|
@ -0,0 +1,39 @@
|
|||||||
|
/**
|
||||||
|
* Copyright 2008 The Apache Software Foundation
|
||||||
|
*
|
||||||
|
* 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;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/** Thrown when the file system needs to be upgraded */
|
||||||
|
public class FileSystemVersionException extends IOException {
|
||||||
|
private static final long serialVersionUID = 1004053363L;
|
||||||
|
|
||||||
|
/** default constructor */
|
||||||
|
public FileSystemVersionException() {
|
||||||
|
super();
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @param s message */
|
||||||
|
public FileSystemVersionException(String s) {
|
||||||
|
super(s);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
@ -30,9 +30,10 @@ import java.util.TreeMap;
|
|||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.client.HConnection;
|
import org.apache.hadoop.hbase.client.HConnection;
|
||||||
import org.apache.hadoop.hbase.client.HConnectionManager;
|
import org.apache.hadoop.hbase.client.HConnectionManager;
|
||||||
import org.apache.hadoop.hbase.client.HTable;
|
import org.apache.hadoop.hbase.client.HTable;
|
||||||
@ -40,25 +41,19 @@ import org.apache.hadoop.hbase.io.BatchUpdate;
|
|||||||
import org.apache.hadoop.hbase.regionserver.HLog;
|
import org.apache.hadoop.hbase.regionserver.HLog;
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.util.Tool;
|
|
||||||
import org.apache.hadoop.util.ToolRunner;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A non-instantiable class that has a static method capable of compacting
|
* A non-instantiable class that has a static method capable of compacting
|
||||||
* a table by merging adjacent regions that have grown too small.
|
* a table by merging adjacent regions.
|
||||||
*/
|
*/
|
||||||
class HMerge implements HConstants, Tool {
|
class HMerge implements HConstants {
|
||||||
static final Log LOG = LogFactory.getLog(HMerge.class);
|
static final Log LOG = LogFactory.getLog(HMerge.class);
|
||||||
static final Random rand = new Random();
|
static final Random rand = new Random();
|
||||||
private Configuration conf;
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Not instantiable
|
* Not instantiable
|
||||||
*/
|
*/
|
||||||
private HMerge() {
|
private HMerge() {}
|
||||||
super();
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Scans the table and merges two adjacent regions if they are small. This
|
* Scans the table and merges two adjacent regions if they are small. This
|
||||||
@ -140,11 +135,6 @@ class HMerge implements HConstants, Tool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected boolean merge(final HRegionInfo[] info) throws IOException {
|
protected boolean merge(final HRegionInfo[] info) throws IOException {
|
||||||
return merge(info, false);
|
|
||||||
}
|
|
||||||
|
|
||||||
protected boolean merge(final HRegionInfo[] info, final boolean force)
|
|
||||||
throws IOException {
|
|
||||||
if(info.length < 2) {
|
if(info.length < 2) {
|
||||||
LOG.info("only one region - nothing to merge");
|
LOG.info("only one region - nothing to merge");
|
||||||
return false;
|
return false;
|
||||||
@ -166,13 +156,13 @@ class HMerge implements HConstants, Tool {
|
|||||||
|
|
||||||
nextSize = nextRegion.largestHStore(midKey).getAggregate();
|
nextSize = nextRegion.largestHStore(midKey).getAggregate();
|
||||||
|
|
||||||
if (force || (currentSize + nextSize) <= (maxFilesize / 2)) {
|
if ((currentSize + nextSize) <= (maxFilesize / 2)) {
|
||||||
// We merge two adjacent regions if their total size is less than
|
// We merge two adjacent regions if their total size is less than
|
||||||
// one half of the desired maximum size
|
// one half of the desired maximum size
|
||||||
LOG.info("merging regions " + currentRegion.getRegionName()
|
LOG.info("merging regions " + currentRegion.getRegionName()
|
||||||
+ " and " + nextRegion.getRegionName());
|
+ " and " + nextRegion.getRegionName());
|
||||||
HRegion mergedRegion =
|
HRegion mergedRegion =
|
||||||
HRegion.closeAndMerge(currentRegion, nextRegion);
|
HRegion.mergeAdjacent(currentRegion, nextRegion);
|
||||||
updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(),
|
updateMeta(currentRegion.getRegionName(), nextRegion.getRegionName(),
|
||||||
mergedRegion);
|
mergedRegion);
|
||||||
break;
|
break;
|
||||||
@ -402,94 +392,4 @@ class HMerge implements HConstants, Tool {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public int run(String[] args) throws Exception {
|
|
||||||
if (args.length == 0 || args.length > 4) {
|
|
||||||
printUsage();
|
|
||||||
return 1;
|
|
||||||
}
|
|
||||||
final String masterPrefix = "--master=";
|
|
||||||
String tableName = null;
|
|
||||||
String loRegion = null;
|
|
||||||
String hiRegion = null;
|
|
||||||
for (int i = 0; i < args.length; i++) {
|
|
||||||
String arg = args[i];
|
|
||||||
if (arg.startsWith(masterPrefix)) {
|
|
||||||
this.conf.set("hbase.master", arg.substring(masterPrefix.length()));
|
|
||||||
} else if (tableName == null) {
|
|
||||||
tableName = arg;
|
|
||||||
continue;
|
|
||||||
} else if (loRegion == null) {
|
|
||||||
loRegion = arg;
|
|
||||||
continue;
|
|
||||||
} else if (hiRegion == null) {
|
|
||||||
hiRegion = arg;
|
|
||||||
continue;
|
|
||||||
} else {
|
|
||||||
throw new IllegalArgumentException("Unsupported argument: " + arg);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// Make finals of the region names so can be refererred to by anonymous
|
|
||||||
// class.
|
|
||||||
final Text lo = new Text(loRegion);
|
|
||||||
final Text hi = new Text(hiRegion);
|
|
||||||
// Make a version of OnlineMerger that does two regions only.
|
|
||||||
Merger m = new OnlineMerger((HBaseConfiguration)this.conf,
|
|
||||||
FileSystem.get(this.conf), new Text(tableName)) {
|
|
||||||
@Override
|
|
||||||
void process() throws IOException {
|
|
||||||
try {
|
|
||||||
for (HRegionInfo[] regionsToMerge = next(); regionsToMerge != null;
|
|
||||||
regionsToMerge = next()) {
|
|
||||||
if (regionsToMerge[0].getRegionName().equals(lo) &&
|
|
||||||
regionsToMerge[1].getRegionName().equals(hi)) {
|
|
||||||
merge(regionsToMerge, true);
|
|
||||||
// Done
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
try {
|
|
||||||
this.hlog.closeAndDelete();
|
|
||||||
} catch(IOException e) {
|
|
||||||
LOG.error(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void checkOfflined(@SuppressWarnings("unused") HRegionInfo hri)
|
|
||||||
throws TableNotDisabledException {
|
|
||||||
// Disabling does not work reliably. Just go ahead and merge.
|
|
||||||
return;
|
|
||||||
}
|
|
||||||
};
|
|
||||||
m.process();
|
|
||||||
return 0;
|
|
||||||
}
|
|
||||||
|
|
||||||
public Configuration getConf() {
|
|
||||||
return this.conf;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setConf(final Configuration c) {
|
|
||||||
this.conf = c;
|
|
||||||
}
|
|
||||||
|
|
||||||
static int printUsage() {
|
|
||||||
System.out.println("merge [--master=MASTER:PORT] <tableName> " +
|
|
||||||
"<lo-region> <hi-region>");
|
|
||||||
System.out.println("Presumes quiescent/offlined table -- does not check");
|
|
||||||
return -1;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run merging of two regions.
|
|
||||||
* @param args
|
|
||||||
* @throws Exception
|
|
||||||
*/
|
|
||||||
public static void main(String[] args) throws Exception {
|
|
||||||
int errCode = ToolRunner.run(new HBaseConfiguration(), new HMerge(), args);
|
|
||||||
System.exit(errCode);
|
|
||||||
}
|
|
||||||
}
|
}
|
@ -25,11 +25,8 @@ import java.util.NoSuchElementException;
|
|||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.hbase.io.HbaseMapWritable;
|
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.io.Writable;
|
|
||||||
import org.apache.hadoop.ipc.RemoteException;
|
import org.apache.hadoop.ipc.RemoteException;
|
||||||
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
import org.apache.hadoop.hbase.ipc.HMasterInterface;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
@ -553,4 +550,17 @@ public class HBaseAdmin implements HConstants {
|
|||||||
Text tableKey = new Text(tableName.toString() + ",,99999999999999");
|
Text tableKey = new Text(tableName.toString() + ",,99999999999999");
|
||||||
return connection.locateRegion(META_TABLE_NAME, tableKey);
|
return connection.locateRegion(META_TABLE_NAME, tableKey);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check to see if HBase is running. Throw an exception if not.
|
||||||
|
*
|
||||||
|
* @param conf
|
||||||
|
* @throws MasterNotRunningException
|
||||||
|
*/
|
||||||
|
public static void checkHBaseAvailable(HBaseConfiguration conf)
|
||||||
|
throws MasterNotRunningException {
|
||||||
|
HBaseConfiguration copyOfConf = new HBaseConfiguration(conf);
|
||||||
|
copyOfConf.setInt("hbase.client.retries.number", 1);
|
||||||
|
new HBaseAdmin(copyOfConf);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -43,6 +43,8 @@ public class Cell implements Writable {
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a new Cell with a given value and timestamp. Used by HStore.
|
* Create a new Cell with a given value and timestamp. Used by HStore.
|
||||||
|
* @param value
|
||||||
|
* @param timestamp
|
||||||
*/
|
*/
|
||||||
public Cell(byte[] value, long timestamp) {
|
public Cell(byte[] value, long timestamp) {
|
||||||
this.value = value;
|
this.value = value;
|
||||||
@ -71,6 +73,7 @@ public class Cell implements Writable {
|
|||||||
// Writable
|
// Writable
|
||||||
//
|
//
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
public void readFields(final DataInput in) throws IOException {
|
public void readFields(final DataInput in) throws IOException {
|
||||||
timestamp = in.readLong();
|
timestamp = in.readLong();
|
||||||
int valueSize = in.readInt();
|
int valueSize = in.readInt();
|
||||||
@ -78,6 +81,7 @@ public class Cell implements Writable {
|
|||||||
in.readFully(value, 0, valueSize);
|
in.readFully(value, 0, valueSize);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
public void write(final DataOutput out) throws IOException {
|
public void write(final DataOutput out) throws IOException {
|
||||||
out.writeLong(timestamp);
|
out.writeLong(timestamp);
|
||||||
out.writeInt(value.length);
|
out.writeInt(value.length);
|
||||||
|
@ -276,9 +276,7 @@ abstract class BaseScanner extends Chore implements HConstants {
|
|||||||
if (!hasReferencesA && !hasReferencesB) {
|
if (!hasReferencesA && !hasReferencesB) {
|
||||||
LOG.info("Deleting region " + parent.getRegionName() +
|
LOG.info("Deleting region " + parent.getRegionName() +
|
||||||
" because daughter splits no longer hold references");
|
" because daughter splits no longer hold references");
|
||||||
if (!HRegion.deleteRegion(master.fs, master.rootdir, parent)) {
|
HRegion.deleteRegion(master.fs, master.rootdir, parent);
|
||||||
LOG.warn("Deletion of " + parent.getRegionName() + " failed");
|
|
||||||
}
|
|
||||||
|
|
||||||
HRegion.removeRegionFromMETA(srvr, metaRegionName,
|
HRegion.removeRegionFromMETA(srvr, metaRegionName,
|
||||||
parent.getRegionName());
|
parent.getRegionName());
|
||||||
|
@ -187,18 +187,7 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
|
|||||||
fs.mkdirs(rootdir);
|
fs.mkdirs(rootdir);
|
||||||
FSUtils.setVersion(fs, rootdir);
|
FSUtils.setVersion(fs, rootdir);
|
||||||
} else {
|
} else {
|
||||||
String fsversion = FSUtils.checkVersion(fs, rootdir);
|
FSUtils.checkVersion(fs, rootdir, true);
|
||||||
if (fsversion == null ||
|
|
||||||
fsversion.compareTo(FILE_SYSTEM_VERSION) != 0) {
|
|
||||||
// Output on stdout so user sees it in terminal.
|
|
||||||
String message = "The HBase data files stored on the FileSystem " +
|
|
||||||
"are from an earlier version of HBase. You need to run " +
|
|
||||||
"'${HBASE_HOME}/bin/hbase migrate' to bring your installation " +
|
|
||||||
"up-to-date.";
|
|
||||||
// Output on stdout so user sees it in terminal.
|
|
||||||
System.out.println("WARNING! " + message + " Master shutting down...");
|
|
||||||
throw new IOException(message);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!fs.exists(rootRegionDir)) {
|
if (!fs.exists(rootRegionDir)) {
|
||||||
@ -262,8 +251,10 @@ public class HMaster extends Thread implements HConstants, HMasterInterface,
|
|||||||
*/
|
*/
|
||||||
protected boolean checkFileSystem() {
|
protected boolean checkFileSystem() {
|
||||||
if (fsOk) {
|
if (fsOk) {
|
||||||
if (!FSUtils.isFileSystemAvailable(fs)) {
|
try {
|
||||||
LOG.fatal("Shutting down HBase cluster: file system not available");
|
FSUtils.checkFileSystemAvailable(fs);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.fatal("Shutting down HBase cluster: file system not available", e);
|
||||||
closed.set(true);
|
closed.set(true);
|
||||||
fsOk = false;
|
fsOk = false;
|
||||||
}
|
}
|
||||||
|
@ -213,7 +213,7 @@ public class HLog implements HConstants {
|
|||||||
*
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
void rollWriter() throws IOException {
|
public void rollWriter() throws IOException {
|
||||||
this.cacheFlushLock.lock();
|
this.cacheFlushLock.lock();
|
||||||
try {
|
try {
|
||||||
if (closed) {
|
if (closed) {
|
||||||
|
@ -38,13 +38,14 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.filter.RowFilterInterface;
|
import org.apache.hadoop.hbase.filter.RowFilterInterface;
|
||||||
import org.apache.hadoop.hbase.io.BatchOperation;
|
import org.apache.hadoop.hbase.io.BatchOperation;
|
||||||
import org.apache.hadoop.hbase.io.BatchUpdate;
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
import org.apache.hadoop.hbase.io.Cell;
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
import org.apache.hadoop.hbase.io.RowResult;
|
|
||||||
import org.apache.hadoop.hbase.util.Writables;
|
import org.apache.hadoop.hbase.util.Writables;
|
||||||
import org.apache.hadoop.io.Text;
|
import org.apache.hadoop.io.Text;
|
||||||
import org.apache.hadoop.io.WritableUtils;
|
import org.apache.hadoop.io.WritableUtils;
|
||||||
@ -106,16 +107,14 @@ public class HRegion implements HConstants {
|
|||||||
final AtomicBoolean closed = new AtomicBoolean(false);
|
final AtomicBoolean closed = new AtomicBoolean(false);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Merge two HRegions. They must be available on the current
|
* Merge two HRegions. The regions must be adjacent andmust not overlap.
|
||||||
* HRegionServer. Returns a brand-new active HRegion, also
|
|
||||||
* running on the current HRegionServer.
|
|
||||||
*
|
*
|
||||||
* @param srcA
|
* @param srcA
|
||||||
* @param srcB
|
* @param srcB
|
||||||
* @return new merged HRegion
|
* @return new merged HRegion
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static HRegion closeAndMerge(final HRegion srcA, final HRegion srcB)
|
public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
HRegion a = srcA;
|
HRegion a = srcA;
|
||||||
@ -123,7 +122,6 @@ public class HRegion implements HConstants {
|
|||||||
|
|
||||||
// Make sure that srcA comes first; important for key-ordering during
|
// Make sure that srcA comes first; important for key-ordering during
|
||||||
// write of the merged file.
|
// write of the merged file.
|
||||||
FileSystem fs = srcA.getFilesystem();
|
|
||||||
if (srcA.getStartKey() == null) {
|
if (srcA.getStartKey() == null) {
|
||||||
if (srcB.getStartKey() == null) {
|
if (srcB.getStartKey() == null) {
|
||||||
throw new IOException("Cannot merge two regions with null start key");
|
throw new IOException("Cannot merge two regions with null start key");
|
||||||
@ -138,28 +136,70 @@ public class HRegion implements HConstants {
|
|||||||
if (! a.getEndKey().equals(b.getStartKey())) {
|
if (! a.getEndKey().equals(b.getStartKey())) {
|
||||||
throw new IOException("Cannot merge non-adjacent regions");
|
throw new IOException("Cannot merge non-adjacent regions");
|
||||||
}
|
}
|
||||||
|
return merge(a, b);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Merge two regions whether they are adjacent or not.
|
||||||
|
*
|
||||||
|
* @param a region a
|
||||||
|
* @param b region b
|
||||||
|
* @return new merged region
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static HRegion merge(HRegion a, HRegion b) throws IOException {
|
||||||
|
if (!a.getRegionInfo().getTableDesc().getName().equals(
|
||||||
|
b.getRegionInfo().getTableDesc().getName())) {
|
||||||
|
throw new IOException("Regions do not belong to the same table");
|
||||||
|
}
|
||||||
|
FileSystem fs = a.getFilesystem();
|
||||||
|
|
||||||
|
// Make sure each region's cache is empty
|
||||||
|
|
||||||
|
a.flushcache();
|
||||||
|
b.flushcache();
|
||||||
|
|
||||||
|
// Compact each region so we only have one store file per family
|
||||||
|
|
||||||
|
a.compactStores();
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Files for region: " + a.getRegionName());
|
||||||
|
listPaths(fs, a.getRegionDir());
|
||||||
|
}
|
||||||
|
b.compactStores();
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Files for region: " + b.getRegionName());
|
||||||
|
listPaths(fs, b.getRegionDir());
|
||||||
|
}
|
||||||
|
|
||||||
HBaseConfiguration conf = a.getConf();
|
HBaseConfiguration conf = a.getConf();
|
||||||
HTableDescriptor tabledesc = a.getTableDesc();
|
HTableDescriptor tabledesc = a.getTableDesc();
|
||||||
HLog log = a.getLog();
|
HLog log = a.getLog();
|
||||||
Path basedir = a.getBaseDir();
|
Path basedir = a.getBaseDir();
|
||||||
Text startKey = a.getStartKey();
|
Text startKey = a.getStartKey().equals(EMPTY_TEXT) ||
|
||||||
Text endKey = b.getEndKey();
|
b.getStartKey().equals(EMPTY_TEXT) ? EMPTY_TEXT :
|
||||||
Path merges = new Path(a.getRegionDir(), MERGEDIR);
|
a.getStartKey().compareTo(b.getStartKey()) <= 0 ?
|
||||||
if(! fs.exists(merges)) {
|
a.getStartKey() : b.getStartKey();
|
||||||
fs.mkdirs(merges);
|
Text endKey = a.getEndKey().equals(EMPTY_TEXT) ||
|
||||||
}
|
b.getEndKey().equals(EMPTY_TEXT) ? EMPTY_TEXT :
|
||||||
|
a.getEndKey().compareTo(b.getEndKey()) <= 0 ?
|
||||||
|
b.getEndKey() : a.getEndKey();
|
||||||
|
|
||||||
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
|
HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey);
|
||||||
Path newRegionDir =
|
LOG.info("Creating new region " + newRegionInfo.toString());
|
||||||
HRegion.getRegionDir(merges, newRegionInfo.getEncodedName());
|
String encodedRegionName = newRegionInfo.getEncodedName();
|
||||||
|
Path newRegionDir = HRegion.getRegionDir(a.getBaseDir(), encodedRegionName);
|
||||||
if(fs.exists(newRegionDir)) {
|
if(fs.exists(newRegionDir)) {
|
||||||
throw new IOException("Cannot merge; target file collision at " +
|
throw new IOException("Cannot merge; target file collision at " +
|
||||||
newRegionDir);
|
newRegionDir);
|
||||||
}
|
}
|
||||||
|
fs.mkdirs(newRegionDir);
|
||||||
|
|
||||||
LOG.info("starting merge of regions: " + a.getRegionName() + " and " +
|
LOG.info("starting merge of regions: " + a.getRegionName() + " and " +
|
||||||
b.getRegionName() + " into new region " + newRegionInfo.toString());
|
b.getRegionName() + " into new region " + newRegionInfo.toString() +
|
||||||
|
" with start key <" + startKey + "> and end key <" + endKey + ">");
|
||||||
|
|
||||||
|
// Move HStoreFiles under new region directory
|
||||||
|
|
||||||
Map<Text, List<HStoreFile>> byFamily =
|
Map<Text, List<HStoreFile>> byFamily =
|
||||||
new TreeMap<Text, List<HStoreFile>>();
|
new TreeMap<Text, List<HStoreFile>>();
|
||||||
@ -167,20 +207,46 @@ public class HRegion implements HConstants {
|
|||||||
byFamily = filesByFamily(byFamily, b.close());
|
byFamily = filesByFamily(byFamily, b.close());
|
||||||
for (Map.Entry<Text, List<HStoreFile>> es : byFamily.entrySet()) {
|
for (Map.Entry<Text, List<HStoreFile>> es : byFamily.entrySet()) {
|
||||||
Text colFamily = es.getKey();
|
Text colFamily = es.getKey();
|
||||||
|
makeColumnFamilyDirs(fs, basedir, encodedRegionName, colFamily, tabledesc);
|
||||||
|
|
||||||
|
// Because we compacted the source regions we should have no more than two
|
||||||
|
// HStoreFiles per family and there will be no reference stores
|
||||||
|
|
||||||
List<HStoreFile> srcFiles = es.getValue();
|
List<HStoreFile> srcFiles = es.getValue();
|
||||||
HStoreFile dst = new HStoreFile(conf, fs, merges,
|
if (srcFiles.size() == 2) {
|
||||||
newRegionInfo.getEncodedName(), colFamily, -1, null);
|
long seqA = srcFiles.get(0).loadInfo(fs);
|
||||||
dst.mergeStoreFiles(srcFiles, fs, conf);
|
long seqB = srcFiles.get(1).loadInfo(fs);
|
||||||
|
if (seqA == seqB) {
|
||||||
|
// We can't have duplicate sequence numbers
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Adjusting sequence number of storeFile " +
|
||||||
|
srcFiles.get(1));
|
||||||
|
}
|
||||||
|
srcFiles.get(1).writeInfo(fs, seqB - 1);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (HStoreFile hsf: srcFiles) {
|
||||||
|
HStoreFile dst = new HStoreFile(conf, fs, basedir,
|
||||||
|
newRegionInfo.getEncodedName(), colFamily, -1, null);
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Renaming " + hsf + " to " + dst);
|
||||||
|
}
|
||||||
|
hsf.rename(fs, dst);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Files for new region");
|
||||||
|
listPaths(fs, newRegionDir);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Done
|
|
||||||
// Construction moves the merge files into place under region.
|
|
||||||
HRegion dstRegion = new HRegion(basedir, log, fs, conf, newRegionInfo,
|
HRegion dstRegion = new HRegion(basedir, log, fs, conf, newRegionInfo,
|
||||||
newRegionDir, null);
|
null, null);
|
||||||
|
dstRegion.compactStores();
|
||||||
// Get rid of merges directory
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Files for new region");
|
||||||
fs.delete(merges);
|
listPaths(fs, dstRegion.getRegionDir());
|
||||||
|
}
|
||||||
|
deleteRegion(fs, a.getRegionDir());
|
||||||
|
deleteRegion(fs, b.getRegionDir());
|
||||||
|
|
||||||
LOG.info("merge completed. New region is " + dstRegion.getRegionName());
|
LOG.info("merge completed. New region is " + dstRegion.getRegionName());
|
||||||
|
|
||||||
@ -206,6 +272,38 @@ public class HRegion implements HConstants {
|
|||||||
return byFamily;
|
return byFamily;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Method to list files in use by region
|
||||||
|
*/
|
||||||
|
static void listFiles(FileSystem fs, HRegion r) throws IOException {
|
||||||
|
listPaths(fs, r.getRegionDir());
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* List the files under the specified directory
|
||||||
|
*
|
||||||
|
* @param fs
|
||||||
|
* @param dir
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private static void listPaths(FileSystem fs, Path dir) throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
FileStatus[] stats = fs.listStatus(dir);
|
||||||
|
if (stats == null || stats.length == 0) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
for (int i = 0; i < stats.length; i++) {
|
||||||
|
String path = stats[i].getPath().toString();
|
||||||
|
if (stats[i].isDir()) {
|
||||||
|
LOG.debug("d " + path);
|
||||||
|
listPaths(fs, stats[i].getPath());
|
||||||
|
} else {
|
||||||
|
LOG.debug("f " + path + " size=" + stats[i].getLen());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
// Members
|
// Members
|
||||||
//////////////////////////////////////////////////////////////////////////////
|
//////////////////////////////////////////////////////////////////////////////
|
||||||
@ -368,8 +466,8 @@ public class HRegion implements HConstants {
|
|||||||
return this.regionInfo;
|
return this.regionInfo;
|
||||||
}
|
}
|
||||||
|
|
||||||
/** returns true if region is closed */
|
/** @return true if region is closed */
|
||||||
boolean isClosed() {
|
public boolean isClosed() {
|
||||||
return this.closed.get();
|
return this.closed.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -409,7 +507,7 @@ public class HRegion implements HConstants {
|
|||||||
final RegionUnavailableListener listener) throws IOException {
|
final RegionUnavailableListener listener) throws IOException {
|
||||||
Text regionName = this.regionInfo.getRegionName();
|
Text regionName = this.regionInfo.getRegionName();
|
||||||
if (isClosed()) {
|
if (isClosed()) {
|
||||||
LOG.info("region " + regionName + " already closed");
|
LOG.warn("region " + regionName + " already closed");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
synchronized (splitLock) {
|
synchronized (splitLock) {
|
||||||
@ -1500,17 +1598,11 @@ public class HRegion implements HConstants {
|
|||||||
|
|
||||||
/** Make sure this is a valid row for the HRegion */
|
/** Make sure this is a valid row for the HRegion */
|
||||||
private void checkRow(Text row) throws IOException {
|
private void checkRow(Text row) throws IOException {
|
||||||
if(((regionInfo.getStartKey().getLength() == 0)
|
if(!rowIsInRange(regionInfo, row)) {
|
||||||
|| (regionInfo.getStartKey().compareTo(row) <= 0))
|
|
||||||
&& ((regionInfo.getEndKey().getLength() == 0)
|
|
||||||
|| (regionInfo.getEndKey().compareTo(row) > 0))) {
|
|
||||||
// all's well
|
|
||||||
|
|
||||||
} else {
|
|
||||||
throw new WrongRegionException("Requested row out of range for " +
|
throw new WrongRegionException("Requested row out of range for " +
|
||||||
"HRegion " + regionInfo.getRegionName() + ", startKey='" +
|
"HRegion " + regionInfo.getRegionName() + ", startKey='" +
|
||||||
regionInfo.getStartKey() + "', getEndKey()='" + regionInfo.getEndKey() +
|
regionInfo.getStartKey() + "', getEndKey()='" +
|
||||||
"', row='" + row + "'");
|
regionInfo.getEndKey() + "', row='" + row + "'");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1825,6 +1917,26 @@ public class HRegion implements HConstants {
|
|||||||
fs, conf, info, null, null);
|
fs, conf, info, null, null);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Convenience method to open a HRegion.
|
||||||
|
* @param info Info for region to be opened.
|
||||||
|
* @param rootDir Root directory for HBase instance
|
||||||
|
* @param log HLog for region to use
|
||||||
|
* @param conf
|
||||||
|
* @return new HRegion
|
||||||
|
*
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static HRegion openHRegion(final HRegionInfo info, final Path rootDir,
|
||||||
|
final HLog log, final HBaseConfiguration conf) throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Opening region: " + info);
|
||||||
|
}
|
||||||
|
return new HRegion(
|
||||||
|
HTableDescriptor.getTableDir(rootDir, info.getTableDesc().getName()),
|
||||||
|
log, FileSystem.get(conf), conf, info, null, null);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Inserts a new region's meta information into the passed
|
* Inserts a new region's meta information into the passed
|
||||||
* <code>meta</code> region. Used by the HMaster bootstrap code adding
|
* <code>meta</code> region. Used by the HMaster bootstrap code adding
|
||||||
@ -1897,23 +2009,25 @@ public class HRegion implements HConstants {
|
|||||||
* @param rootdir qualified path of HBase root directory
|
* @param rootdir qualified path of HBase root directory
|
||||||
* @param info HRegionInfo for region to be deleted
|
* @param info HRegionInfo for region to be deleted
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @return True if deleted.
|
|
||||||
*/
|
*/
|
||||||
public static boolean deleteRegion(FileSystem fs, Path rootdir,
|
public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
|
||||||
HRegionInfo info)
|
|
||||||
throws IOException {
|
throws IOException {
|
||||||
Path p = HRegion.getRegionDir(rootdir, info);
|
deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
|
||||||
if (LOG.isDebugEnabled()) {
|
|
||||||
LOG.debug("DELETING region " + p.toString());
|
|
||||||
}
|
}
|
||||||
return fs.delete(p);
|
|
||||||
|
private static void deleteRegion(FileSystem fs, Path regiondir)
|
||||||
|
throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("DELETING region " + regiondir.toString());
|
||||||
|
}
|
||||||
|
FileUtil.fullyDelete(fs, regiondir);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Computes the Path of the HRegion
|
* Computes the Path of the HRegion
|
||||||
*
|
*
|
||||||
* @param tabledir qualified path for table
|
* @param tabledir qualified path for table
|
||||||
* @param name region file name ENCODED!
|
* @param name ENCODED region name
|
||||||
* @return Path of HRegion directory
|
* @return Path of HRegion directory
|
||||||
* @see HRegionInfo#encodeRegionName(Text)
|
* @see HRegionInfo#encodeRegionName(Text)
|
||||||
*/
|
*/
|
||||||
@ -1934,4 +2048,40 @@ public class HRegion implements HConstants {
|
|||||||
info.getEncodedName()
|
info.getEncodedName()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Determines if the specified row is within the row range specified by the
|
||||||
|
* specified HRegionInfo
|
||||||
|
*
|
||||||
|
* @param info HRegionInfo that specifies the row range
|
||||||
|
* @param row row to be checked
|
||||||
|
* @return true if the row is within the range specified by the HRegionInfo
|
||||||
|
*/
|
||||||
|
public static boolean rowIsInRange(HRegionInfo info, Text row) {
|
||||||
|
return ((info.getStartKey().getLength() == 0) ||
|
||||||
|
(info.getStartKey().compareTo(row) <= 0)) &&
|
||||||
|
((info.getEndKey().getLength() == 0) ||
|
||||||
|
(info.getEndKey().compareTo(row) > 0));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Make the directories for a specific column family
|
||||||
|
*
|
||||||
|
* @param fs the file system
|
||||||
|
* @param basedir base directory where region will live (usually the table dir)
|
||||||
|
* @param encodedRegionName encoded region name
|
||||||
|
* @param colFamily the column family
|
||||||
|
* @param tabledesc table descriptor of table
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void makeColumnFamilyDirs(FileSystem fs, Path basedir,
|
||||||
|
String encodedRegionName, Text colFamily, HTableDescriptor tabledesc)
|
||||||
|
throws IOException {
|
||||||
|
fs.mkdirs(HStoreFile.getMapDir(basedir, encodedRegionName, colFamily));
|
||||||
|
fs.mkdirs(HStoreFile.getInfoDir(basedir, encodedRegionName, colFamily));
|
||||||
|
if (tabledesc.families().get(new Text(colFamily + ":")).getBloomFilter() !=
|
||||||
|
null) {
|
||||||
|
fs.mkdirs(HStoreFile.getFilterDir(basedir, encodedRegionName, colFamily));
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
@ -71,7 +71,6 @@ import org.apache.hadoop.hbase.io.BatchUpdate;
|
|||||||
import org.apache.hadoop.hbase.io.Cell;
|
import org.apache.hadoop.hbase.io.Cell;
|
||||||
import org.apache.hadoop.hbase.io.RowResult;
|
import org.apache.hadoop.hbase.io.RowResult;
|
||||||
import org.apache.hadoop.hbase.io.HbaseMapWritable;
|
import org.apache.hadoop.hbase.io.HbaseMapWritable;
|
||||||
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
|
||||||
import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HMasterRegionInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
import org.apache.hadoop.hbase.ipc.HRegionInterface;
|
||||||
import org.apache.hadoop.hbase.ipc.HbaseRPC;
|
import org.apache.hadoop.hbase.ipc.HbaseRPC;
|
||||||
@ -1179,21 +1178,18 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/** @return the info server */
|
/** @return the info server */
|
||||||
/**
|
|
||||||
* Get the InfoServer this HRegionServer has put up.
|
|
||||||
*/
|
|
||||||
public InfoServer getInfoServer() {
|
public InfoServer getInfoServer() {
|
||||||
return infoServer;
|
return infoServer;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if a stop has been requested.
|
* @return true if a stop has been requested.
|
||||||
*/
|
*/
|
||||||
public boolean isStopRequested() {
|
public boolean isStopRequested() {
|
||||||
return stopRequested.get();
|
return stopRequested.get();
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Get the write lock for the server */
|
/** @return the write lock for the server */
|
||||||
ReentrantReadWriteLock.WriteLock getWriteLock() {
|
ReentrantReadWriteLock.WriteLock getWriteLock() {
|
||||||
return lock.writeLock();
|
return lock.writeLock();
|
||||||
}
|
}
|
||||||
@ -1295,17 +1291,11 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable {
|
|||||||
* @return false if file system is not available
|
* @return false if file system is not available
|
||||||
*/
|
*/
|
||||||
protected boolean checkFileSystem() {
|
protected boolean checkFileSystem() {
|
||||||
if (this.fsOk) {
|
if (this.fsOk && fs != null) {
|
||||||
try {
|
try {
|
||||||
if (fs != null && !FSUtils.isFileSystemAvailable(fs)) {
|
FSUtils.checkFileSystemAvailable(fs);
|
||||||
LOG.fatal("Shutting down HRegionServer: file system not available");
|
} catch (IOException e) {
|
||||||
this.abortRequested = true;
|
LOG.fatal("Shutting down HRegionServer: file system not available", e);
|
||||||
this.stopRequested.set(true);
|
|
||||||
fsOk = false;
|
|
||||||
}
|
|
||||||
} catch (Exception e) {
|
|
||||||
LOG.error("Failed get of filesystem", e);
|
|
||||||
LOG.fatal("Shutting down HRegionServer: file system not available");
|
|
||||||
this.abortRequested = true;
|
this.abortRequested = true;
|
||||||
this.stopRequested.set(true);
|
this.stopRequested.set(true);
|
||||||
fsOk = false;
|
fsOk = false;
|
||||||
|
@ -28,7 +28,6 @@ import java.io.DataOutputStream;
|
|||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UnsupportedEncodingException;
|
import java.io.UnsupportedEncodingException;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
@ -284,51 +283,6 @@ public class HStoreFile implements HConstants {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Merges the contents of the given source HStoreFiles into a single new one.
|
|
||||||
*
|
|
||||||
* @param srcFiles files to be merged
|
|
||||||
* @param fs file system
|
|
||||||
* @param conf configuration object
|
|
||||||
* @throws IOException
|
|
||||||
*/
|
|
||||||
void mergeStoreFiles(List<HStoreFile> srcFiles, FileSystem fs,
|
|
||||||
@SuppressWarnings("hiding") Configuration conf)
|
|
||||||
throws IOException {
|
|
||||||
// Copy all the source MapFile tuples into this HSF's MapFile
|
|
||||||
MapFile.Writer out = new MapFile.Writer(conf, fs,
|
|
||||||
getMapFilePath().toString(),
|
|
||||||
HStoreKey.class, ImmutableBytesWritable.class);
|
|
||||||
|
|
||||||
try {
|
|
||||||
for(HStoreFile src: srcFiles) {
|
|
||||||
MapFile.Reader in = src.getReader(fs, null);
|
|
||||||
try {
|
|
||||||
HStoreKey readkey = new HStoreKey();
|
|
||||||
ImmutableBytesWritable readval = new ImmutableBytesWritable();
|
|
||||||
while(in.next(readkey, readval)) {
|
|
||||||
out.append(readkey, readval);
|
|
||||||
}
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
in.close();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
out.close();
|
|
||||||
}
|
|
||||||
// Build a unified InfoFile from the source InfoFiles.
|
|
||||||
|
|
||||||
long unifiedSeqId = -1;
|
|
||||||
for(HStoreFile hsf: srcFiles) {
|
|
||||||
long curSeqId = hsf.loadInfo(fs);
|
|
||||||
if(curSeqId > unifiedSeqId) {
|
|
||||||
unifiedSeqId = curSeqId;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
writeInfo(fs, unifiedSeqId);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Reads in an info file
|
* Reads in an info file
|
||||||
*
|
*
|
||||||
|
@ -24,12 +24,16 @@ import java.io.IOException;
|
|||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
|
import org.apache.hadoop.dfs.DistributedFileSystem;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.FileSystemVersionException;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.dfs.DistributedFileSystem;
|
import org.apache.hadoop.hbase.RemoteExceptionHandler;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Utility methods for interacting with the underlying file system.
|
* Utility methods for interacting with the underlying file system.
|
||||||
@ -46,34 +50,32 @@ public class FSUtils {
|
|||||||
* Checks to see if the specified file system is available
|
* Checks to see if the specified file system is available
|
||||||
*
|
*
|
||||||
* @param fs
|
* @param fs
|
||||||
* @return true if the specified file system is available.
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static boolean isFileSystemAvailable(final FileSystem fs) {
|
public static void checkFileSystemAvailable(final FileSystem fs)
|
||||||
|
throws IOException {
|
||||||
if (!(fs instanceof DistributedFileSystem)) {
|
if (!(fs instanceof DistributedFileSystem)) {
|
||||||
return true;
|
return;
|
||||||
}
|
}
|
||||||
String exception = "";
|
IOException exception = null;
|
||||||
boolean available = false;
|
|
||||||
DistributedFileSystem dfs = (DistributedFileSystem) fs;
|
DistributedFileSystem dfs = (DistributedFileSystem) fs;
|
||||||
try {
|
try {
|
||||||
if (dfs.exists(new Path("/"))) {
|
if (dfs.exists(new Path("/"))) {
|
||||||
available = true;
|
return;
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
exception = e.getMessage();
|
exception = RemoteExceptionHandler.checkIOException(e);
|
||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
if (!available) {
|
|
||||||
LOG.fatal("File system is not available.. Thread: " +
|
|
||||||
Thread.currentThread().getName() + ": " + exception);
|
|
||||||
fs.close();
|
fs.close();
|
||||||
}
|
|
||||||
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.error("file system close failed: ", e);
|
LOG.error("file system close failed: ", e);
|
||||||
}
|
}
|
||||||
return available;
|
IOException io = new IOException("File system is not available");
|
||||||
|
io.initCause(exception);
|
||||||
|
throw io;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -84,18 +86,46 @@ public class FSUtils {
|
|||||||
* @return null if no version file exists, version string otherwise.
|
* @return null if no version file exists, version string otherwise.
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
public static String checkVersion(FileSystem fs, Path rootdir) throws IOException {
|
public static String getVersion(FileSystem fs, Path rootdir)
|
||||||
|
throws IOException {
|
||||||
Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
|
Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
|
||||||
String version = null;
|
String version = null;
|
||||||
if (fs.exists(versionFile)) {
|
if (fs.exists(versionFile)) {
|
||||||
FSDataInputStream s =
|
FSDataInputStream s =
|
||||||
fs.open(new Path(rootdir, HConstants.VERSION_FILE_NAME));
|
fs.open(new Path(rootdir, HConstants.VERSION_FILE_NAME));
|
||||||
|
try {
|
||||||
version = DataInputStream.readUTF(s);
|
version = DataInputStream.readUTF(s);
|
||||||
|
} finally {
|
||||||
s.close();
|
s.close();
|
||||||
}
|
}
|
||||||
|
}
|
||||||
return version;
|
return version;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies current version of file system
|
||||||
|
*
|
||||||
|
* @param fs file system
|
||||||
|
* @param rootdir root directory of HBase installation
|
||||||
|
* @param message if true, issues a message on System.out
|
||||||
|
*
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void checkVersion(FileSystem fs, Path rootdir, boolean message)
|
||||||
|
throws IOException {
|
||||||
|
String version = getVersion(fs, rootdir);
|
||||||
|
if (version == null ||
|
||||||
|
version.compareTo(HConstants.FILE_SYSTEM_VERSION) != 0) {
|
||||||
|
// Output on stdout so user sees it in terminal.
|
||||||
|
String msg = "File system needs to be upgraded. Run " +
|
||||||
|
"the '${HBASE_HOME}/bin/hbase migrate' script.";
|
||||||
|
if (message) {
|
||||||
|
System.out.println("WARNING! " + msg);
|
||||||
|
}
|
||||||
|
throw new FileSystemVersionException(msg);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Sets version of file system
|
* Sets version of file system
|
||||||
*
|
*
|
||||||
|
@ -20,6 +20,9 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.hbase.util;
|
package org.apache.hadoop.hbase.util;
|
||||||
|
|
||||||
|
import java.io.FileInputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* lookup3.c, by Bob Jenkins, May 2006, Public Domain.
|
* lookup3.c, by Bob Jenkins, May 2006, Public Domain.
|
||||||
* <a href="http://burtleburtle.net/bob/c/lookup3.c">lookup3.c</a>
|
* <a href="http://burtleburtle.net/bob/c/lookup3.c">lookup3.c</a>
|
||||||
@ -231,4 +234,23 @@ public class JenkinsHash {
|
|||||||
|
|
||||||
return Long.valueOf(c & INT_MASK).intValue();
|
return Long.valueOf(c & INT_MASK).intValue();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Compute the hash of the specified file
|
||||||
|
* @param args name of file to compute hash of.
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public static void main(String[] args) throws IOException {
|
||||||
|
if (args.length != 1) {
|
||||||
|
System.err.println("Usage: JenkinsHash filename");
|
||||||
|
System.exit(-1);
|
||||||
|
}
|
||||||
|
FileInputStream in = new FileInputStream(args[0]);
|
||||||
|
byte[] bytes = new byte[512];
|
||||||
|
int value = 0;
|
||||||
|
for (int length = in.read(bytes); length > 0 ; length = in.read(bytes)) {
|
||||||
|
value = hash(bytes, length, value);
|
||||||
|
}
|
||||||
|
System.out.println(Math.abs(value));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
373
src/java/org/apache/hadoop/hbase/util/Merge.java
Normal file
373
src/java/org/apache/hadoop/hbase/util/Merge.java
Normal file
@ -0,0 +1,373 @@
|
|||||||
|
/**
|
||||||
|
* Copyright 2008 The Apache Software Foundation
|
||||||
|
*
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.conf.Configured;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.io.WritableComparator;
|
||||||
|
import org.apache.hadoop.util.GenericOptionsParser;
|
||||||
|
import org.apache.hadoop.util.Tool;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.client.HBaseAdmin;
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HLog;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility that can merge any two regions in the same table: adjacent,
|
||||||
|
* overlapping or disjoint.
|
||||||
|
*/
|
||||||
|
public class Merge extends Configured implements Tool {
|
||||||
|
static final Log LOG = LogFactory.getLog(Merge.class);
|
||||||
|
private final HBaseConfiguration conf;
|
||||||
|
private Path rootdir;
|
||||||
|
private volatile MetaUtils utils;
|
||||||
|
private Text tableName; // Name of table
|
||||||
|
private volatile Text region1; // Name of region 1
|
||||||
|
private volatile Text region2; // Name of region 2
|
||||||
|
private volatile boolean isMetaTable;
|
||||||
|
private volatile HRegionInfo mergeInfo;
|
||||||
|
|
||||||
|
/** default constructor */
|
||||||
|
public Merge() {
|
||||||
|
this(new HBaseConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param conf
|
||||||
|
*/
|
||||||
|
public Merge(HBaseConfiguration conf) {
|
||||||
|
super(conf);
|
||||||
|
this.conf = conf;
|
||||||
|
this.mergeInfo = null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public int run(String[] args) throws Exception {
|
||||||
|
if (parseArgs(args) != 0) {
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Verify file system is up.
|
||||||
|
FileSystem fs = FileSystem.get(this.conf); // get DFS handle
|
||||||
|
LOG.info("Verifying that file system is available...");
|
||||||
|
try {
|
||||||
|
FSUtils.checkFileSystemAvailable(fs);
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.fatal("File system is not available", e);
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Verify HBase is down
|
||||||
|
LOG.info("Verifying that HBase is not running...");
|
||||||
|
try {
|
||||||
|
HBaseAdmin.checkHBaseAvailable(conf);
|
||||||
|
LOG.fatal("HBase cluster must be off-line.");
|
||||||
|
return -1;
|
||||||
|
} catch (MasterNotRunningException e) {
|
||||||
|
// Expected. Ignore.
|
||||||
|
}
|
||||||
|
|
||||||
|
// Initialize MetaUtils and and get the root of the HBase installation
|
||||||
|
|
||||||
|
this.utils = new MetaUtils(conf);
|
||||||
|
this.rootdir = utils.initialize();
|
||||||
|
try {
|
||||||
|
if (isMetaTable) {
|
||||||
|
mergeTwoMetaRegions();
|
||||||
|
} else {
|
||||||
|
mergeTwoRegions();
|
||||||
|
}
|
||||||
|
return 0;
|
||||||
|
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.fatal("Merge failed", e);
|
||||||
|
utils.scanMetaRegion(HRegionInfo.firstMetaRegionInfo,
|
||||||
|
new MetaUtils.ScannerListener() {
|
||||||
|
public boolean processRow(HRegionInfo info) {
|
||||||
|
System.err.println(info.toString());
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
return -1;
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
if (this.utils != null && this.utils.isInitialized()) {
|
||||||
|
this.utils.shutdown();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return HRegionInfo for merge result */
|
||||||
|
HRegionInfo getMergedHRegionInfo() {
|
||||||
|
return this.mergeInfo;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Merge two meta regions. This is unlikely to be needed soon as we have only
|
||||||
|
* seend the meta table split once and that was with 64MB regions. With 256MB
|
||||||
|
* regions, it will be some time before someone has enough data in HBase to
|
||||||
|
* split the meta region and even less likely that a merge of two meta
|
||||||
|
* regions will be needed, but it is included for completeness.
|
||||||
|
*/
|
||||||
|
private void mergeTwoMetaRegions() throws IOException {
|
||||||
|
HRegion rootRegion = utils.getRootRegion();
|
||||||
|
HRegionInfo info1 = Writables.getHRegionInfoOrNull(
|
||||||
|
rootRegion.get(region1, HConstants.COL_REGIONINFO).getValue());
|
||||||
|
HRegionInfo info2 = Writables.getHRegionInfoOrNull(
|
||||||
|
rootRegion.get(region2, HConstants.COL_REGIONINFO).getValue());
|
||||||
|
HRegion merged = merge(info1, rootRegion, info2, rootRegion);
|
||||||
|
LOG.info("Adding " + merged.getRegionInfo() + " to " +
|
||||||
|
rootRegion.getRegionInfo());
|
||||||
|
HRegion.addRegionToMETA(rootRegion, merged);
|
||||||
|
merged.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class MetaScannerListener
|
||||||
|
implements MetaUtils.ScannerListener {
|
||||||
|
private final Text region1;
|
||||||
|
private final Text region2;
|
||||||
|
private HRegionInfo meta1 = null;
|
||||||
|
private HRegionInfo meta2 = null;
|
||||||
|
|
||||||
|
MetaScannerListener(Text region1, Text region2) {
|
||||||
|
this.region1 = region1;
|
||||||
|
this.region2 = region2;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
public boolean processRow(HRegionInfo info) {
|
||||||
|
if (meta1 == null && HRegion.rowIsInRange(info, region1)) {
|
||||||
|
meta1 = info;
|
||||||
|
}
|
||||||
|
if (region2 != null && meta2 == null &&
|
||||||
|
HRegion.rowIsInRange(info, region2)) {
|
||||||
|
meta2 = info;
|
||||||
|
}
|
||||||
|
return meta1 == null || (region2 != null && meta2 == null);
|
||||||
|
}
|
||||||
|
|
||||||
|
HRegionInfo getMeta1() {
|
||||||
|
return meta1;
|
||||||
|
}
|
||||||
|
|
||||||
|
HRegionInfo getMeta2() {
|
||||||
|
return meta2;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Merges two regions from a user table.
|
||||||
|
*/
|
||||||
|
private void mergeTwoRegions() throws IOException {
|
||||||
|
// Scan the root region for all the meta regions that contain the regions
|
||||||
|
// we're merging.
|
||||||
|
MetaScannerListener listener = new MetaScannerListener(region1, region2);
|
||||||
|
utils.scanRootRegion(listener);
|
||||||
|
HRegionInfo meta1 = listener.getMeta1();
|
||||||
|
if (meta1 == null) {
|
||||||
|
throw new IOException("Could not find meta region for " + region1);
|
||||||
|
}
|
||||||
|
HRegionInfo meta2 = listener.getMeta2();
|
||||||
|
if (meta2 == null) {
|
||||||
|
throw new IOException("Could not find meta region for " + region2);
|
||||||
|
}
|
||||||
|
|
||||||
|
HRegion metaRegion1 = utils.getMetaRegion(meta1);
|
||||||
|
HRegionInfo info1 = Writables.getHRegionInfoOrNull(
|
||||||
|
metaRegion1.get(region1, HConstants.COL_REGIONINFO).getValue());
|
||||||
|
|
||||||
|
|
||||||
|
HRegion metaRegion2 = null;
|
||||||
|
if (meta1.getRegionName().equals(meta2.getRegionName())) {
|
||||||
|
metaRegion2 = metaRegion1;
|
||||||
|
} else {
|
||||||
|
metaRegion2 = utils.getMetaRegion(meta2);
|
||||||
|
}
|
||||||
|
HRegionInfo info2 = Writables.getHRegionInfoOrNull(
|
||||||
|
metaRegion2.get(region2, HConstants.COL_REGIONINFO).getValue());
|
||||||
|
|
||||||
|
HRegion merged = merge(info1, metaRegion1, info2, metaRegion2);
|
||||||
|
|
||||||
|
// Now find the meta region which will contain the newly merged region
|
||||||
|
|
||||||
|
listener = new MetaScannerListener(merged.getRegionName(), null);
|
||||||
|
utils.scanRootRegion(listener);
|
||||||
|
HRegionInfo mergedInfo = listener.getMeta1();
|
||||||
|
if (mergedInfo == null) {
|
||||||
|
throw new IOException("Could not find meta region for " +
|
||||||
|
merged.getRegionName());
|
||||||
|
}
|
||||||
|
HRegion mergeMeta = null;
|
||||||
|
if (mergedInfo.getRegionName().equals(meta1.getRegionName())) {
|
||||||
|
mergeMeta = metaRegion1;
|
||||||
|
} else if (mergedInfo.getRegionName().equals(meta2.getRegionName())) {
|
||||||
|
mergeMeta = metaRegion2;
|
||||||
|
} else {
|
||||||
|
mergeMeta = utils.getMetaRegion(mergedInfo);
|
||||||
|
}
|
||||||
|
LOG.info("Adding " + merged.getRegionInfo() + " to " +
|
||||||
|
mergeMeta.getRegionInfo());
|
||||||
|
|
||||||
|
HRegion.addRegionToMETA(mergeMeta, merged);
|
||||||
|
merged.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Actually merge two regions and update their info in the meta region(s)
|
||||||
|
* If the meta is split, meta1 may be different from meta2. (and we may have
|
||||||
|
* to scan the meta if the resulting merged region does not go in either)
|
||||||
|
* Returns HRegion object for newly merged region
|
||||||
|
*/
|
||||||
|
private HRegion merge(HRegionInfo info1, HRegion meta1, HRegionInfo info2,
|
||||||
|
HRegion meta2) throws IOException {
|
||||||
|
if (info1 == null) {
|
||||||
|
throw new IOException("Could not find " + region1 + " in " +
|
||||||
|
meta1.getRegionName());
|
||||||
|
}
|
||||||
|
if (info2 == null) {
|
||||||
|
throw new IOException("Cound not find " + region2 + " in " +
|
||||||
|
meta2.getRegionName());
|
||||||
|
}
|
||||||
|
HRegion merged = null;
|
||||||
|
HLog log = utils.getLog();
|
||||||
|
HRegion region1 =
|
||||||
|
HRegion.openHRegion(info1, this.rootdir, log, this.conf);
|
||||||
|
try {
|
||||||
|
HRegion region2 =
|
||||||
|
HRegion.openHRegion(info2, this.rootdir, log, this.conf);
|
||||||
|
try {
|
||||||
|
merged = HRegion.merge(region1, region2);
|
||||||
|
} finally {
|
||||||
|
if (!region2.isClosed()) {
|
||||||
|
region2.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
if (!region1.isClosed()) {
|
||||||
|
region1.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Remove the old regions from meta.
|
||||||
|
// HRegion.merge has already deleted their files
|
||||||
|
|
||||||
|
removeRegionFromMeta(meta1, info1);
|
||||||
|
removeRegionFromMeta(meta2, info2);
|
||||||
|
|
||||||
|
this.mergeInfo = merged.getRegionInfo();
|
||||||
|
return merged;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Removes a region's meta information from the passed <code>meta</code>
|
||||||
|
* region.
|
||||||
|
*
|
||||||
|
* @param meta META HRegion to be updated
|
||||||
|
* @param regioninfo HRegionInfo of region to remove from <code>meta</code>
|
||||||
|
*
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private void removeRegionFromMeta(HRegion meta, HRegionInfo regioninfo)
|
||||||
|
throws IOException {
|
||||||
|
if (LOG.isDebugEnabled()) {
|
||||||
|
LOG.debug("Removing region: " + regioninfo + " from " + meta);
|
||||||
|
}
|
||||||
|
meta.deleteAll(regioninfo.getRegionName(), System.currentTimeMillis());
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Adds a region's meta information from the passed <code>meta</code>
|
||||||
|
* region.
|
||||||
|
*
|
||||||
|
* @param metainfo META HRegionInfo to be updated
|
||||||
|
* @param region HRegion to add to <code>meta</code>
|
||||||
|
*
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
private int parseArgs(String[] args) {
|
||||||
|
GenericOptionsParser parser =
|
||||||
|
new GenericOptionsParser(this.getConf(), args);
|
||||||
|
|
||||||
|
String[] remainingArgs = parser.getRemainingArgs();
|
||||||
|
if (remainingArgs.length != 3) {
|
||||||
|
usage();
|
||||||
|
return -1;
|
||||||
|
}
|
||||||
|
tableName = new Text(remainingArgs[0]);
|
||||||
|
isMetaTable = tableName.compareTo(HConstants.META_TABLE_NAME) == 0;
|
||||||
|
|
||||||
|
region1 = new Text(remainingArgs[1]);
|
||||||
|
region2 = new Text(remainingArgs[2]);
|
||||||
|
int status = 0;
|
||||||
|
if (WritableComparator.compareBytes(
|
||||||
|
tableName.getBytes(), 0, tableName.getLength(),
|
||||||
|
region1.getBytes(), 0, tableName.getLength()) != 0) {
|
||||||
|
LOG.error("Region " + region1 + " does not belong to table " + tableName);
|
||||||
|
status = -1;
|
||||||
|
}
|
||||||
|
if (WritableComparator.compareBytes(
|
||||||
|
tableName.getBytes(), 0, tableName.getLength(),
|
||||||
|
region2.getBytes(), 0, tableName.getLength()) != 0) {
|
||||||
|
LOG.error("Region " + region2 + " does not belong to table " + tableName);
|
||||||
|
status = -1;
|
||||||
|
}
|
||||||
|
if (region1.equals(region2)) {
|
||||||
|
LOG.error("Can't merge a region with itself");
|
||||||
|
status = -1;
|
||||||
|
}
|
||||||
|
return status;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void usage() {
|
||||||
|
System.err.println(
|
||||||
|
"Usage: bin/hbase merge <table-name> <region-1> <region-2>\n");
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Main program
|
||||||
|
*
|
||||||
|
* @param args
|
||||||
|
*/
|
||||||
|
public static void main(String[] args) {
|
||||||
|
int status = 0;
|
||||||
|
try {
|
||||||
|
status = ToolRunner.run(new Merge(), args);
|
||||||
|
} catch (Exception e) {
|
||||||
|
LOG.error("exiting due to error", e);
|
||||||
|
status = -1;
|
||||||
|
}
|
||||||
|
System.exit(status);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
298
src/java/org/apache/hadoop/hbase/util/MetaUtils.java
Normal file
298
src/java/org/apache/hadoop/hbase/util/MetaUtils.java
Normal file
@ -0,0 +1,298 @@
|
|||||||
|
/**
|
||||||
|
* Copyright 2008 The Apache Software Foundation
|
||||||
|
*
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import java.io.FileNotFoundException;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.concurrent.ConcurrentHashMap;
|
||||||
|
import java.util.SortedMap;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
|
||||||
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HLog;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.HScannerInterface;
|
||||||
|
import org.apache.hadoop.hbase.HStoreKey;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Contains utility methods for manipulating HBase meta tables
|
||||||
|
*/
|
||||||
|
public class MetaUtils {
|
||||||
|
private static final Log LOG = LogFactory.getLog(MetaUtils.class);
|
||||||
|
|
||||||
|
private final HBaseConfiguration conf;
|
||||||
|
boolean initialized;
|
||||||
|
private FileSystem fs;
|
||||||
|
private Path rootdir;
|
||||||
|
private HLog log;
|
||||||
|
private HRegion rootRegion;
|
||||||
|
private ConcurrentHashMap<Text, HRegion> metaRegions;
|
||||||
|
|
||||||
|
/** Default constructor */
|
||||||
|
public MetaUtils() {
|
||||||
|
this(new HBaseConfiguration());
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @param conf HBaseConfiguration */
|
||||||
|
public MetaUtils(HBaseConfiguration conf) {
|
||||||
|
this.conf = conf;
|
||||||
|
conf.setInt("hbase.client.retries.number", 1);
|
||||||
|
this.initialized = false;
|
||||||
|
this.rootRegion = null;
|
||||||
|
this.metaRegions = new ConcurrentHashMap<Text, HRegion>();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies that DFS is available and that HBase is off-line.
|
||||||
|
*
|
||||||
|
* @return Path of root directory of HBase installation
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public Path initialize() throws IOException {
|
||||||
|
if (!initialized) {
|
||||||
|
this.fs = FileSystem.get(this.conf); // get DFS handle
|
||||||
|
|
||||||
|
// Get root directory of HBase installation
|
||||||
|
|
||||||
|
this.rootdir =
|
||||||
|
fs.makeQualified(new Path(this.conf.get(HConstants.HBASE_DIR)));
|
||||||
|
|
||||||
|
if (!fs.exists(rootdir)) {
|
||||||
|
String message = "HBase root directory " + rootdir.toString() +
|
||||||
|
" does not exist.";
|
||||||
|
LOG.error(message);
|
||||||
|
throw new FileNotFoundException(message);
|
||||||
|
}
|
||||||
|
|
||||||
|
this.log = new HLog(this.fs,
|
||||||
|
new Path(this.fs.getHomeDirectory(),
|
||||||
|
HConstants.HREGION_LOGDIR_NAME + "_" + System.currentTimeMillis()
|
||||||
|
),
|
||||||
|
this.conf, null
|
||||||
|
);
|
||||||
|
|
||||||
|
this.initialized = true;
|
||||||
|
}
|
||||||
|
return this.rootdir;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return true if initialization completed successfully */
|
||||||
|
public boolean isInitialized() {
|
||||||
|
return this.initialized;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @return the HLog */
|
||||||
|
public HLog getLog() {
|
||||||
|
if (!initialized) {
|
||||||
|
throw new IllegalStateException("Must call initialize method first.");
|
||||||
|
}
|
||||||
|
return this.log;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return HRegion for root region
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public HRegion getRootRegion() throws IOException {
|
||||||
|
if (!initialized) {
|
||||||
|
throw new IllegalStateException("Must call initialize method first.");
|
||||||
|
}
|
||||||
|
if (this.rootRegion == null) {
|
||||||
|
openRootRegion();
|
||||||
|
}
|
||||||
|
return this.rootRegion;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Open or return cached opened meta region
|
||||||
|
*
|
||||||
|
* @param metaInfo HRegionInfo for meta region
|
||||||
|
* @return meta HRegion
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public HRegion getMetaRegion(HRegionInfo metaInfo) throws IOException {
|
||||||
|
if (!initialized) {
|
||||||
|
throw new IllegalStateException("Must call initialize method first.");
|
||||||
|
}
|
||||||
|
HRegion meta = metaRegions.get(metaInfo.getRegionName());
|
||||||
|
if (meta == null) {
|
||||||
|
meta = openMetaRegion(metaInfo);
|
||||||
|
metaRegions.put(metaInfo.getRegionName(), meta);
|
||||||
|
}
|
||||||
|
return meta;
|
||||||
|
}
|
||||||
|
|
||||||
|
/** Closes root region if open. Also closes and deletes the HLog. */
|
||||||
|
public void shutdown() {
|
||||||
|
if (this.rootRegion != null) {
|
||||||
|
try {
|
||||||
|
this.rootRegion.close();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("closing root region", e);
|
||||||
|
} finally {
|
||||||
|
this.rootRegion = null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
for (HRegion r: metaRegions.values()) {
|
||||||
|
r.close();
|
||||||
|
}
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("closing meta region", e);
|
||||||
|
} finally {
|
||||||
|
metaRegions.clear();
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
this.log.rollWriter();
|
||||||
|
this.log.closeAndDelete();
|
||||||
|
} catch (IOException e) {
|
||||||
|
LOG.error("closing HLog", e);
|
||||||
|
} finally {
|
||||||
|
this.log = null;
|
||||||
|
}
|
||||||
|
this.initialized = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Used by scanRootRegion and scanMetaRegion to call back the caller so it
|
||||||
|
* can process the data for a row.
|
||||||
|
*/
|
||||||
|
public interface ScannerListener {
|
||||||
|
/**
|
||||||
|
* Callback so client of scanner can process row contents
|
||||||
|
*
|
||||||
|
* @param info HRegionInfo for row
|
||||||
|
* @return false to terminate the scan
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public boolean processRow(HRegionInfo info) throws IOException;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scans the root region. For every meta region found, calls the listener with
|
||||||
|
* the HRegionInfo of the meta region.
|
||||||
|
*
|
||||||
|
* @param listener method to be called for each meta region found
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void scanRootRegion(ScannerListener listener) throws IOException {
|
||||||
|
if (!initialized) {
|
||||||
|
throw new IllegalStateException("Must call initialize method first.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Open root region so we can scan it
|
||||||
|
|
||||||
|
if (this.rootRegion == null) {
|
||||||
|
openRootRegion();
|
||||||
|
}
|
||||||
|
|
||||||
|
HScannerInterface rootScanner = rootRegion.getScanner(
|
||||||
|
HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
|
||||||
|
HConstants.LATEST_TIMESTAMP, null);
|
||||||
|
|
||||||
|
try {
|
||||||
|
HStoreKey key = new HStoreKey();
|
||||||
|
SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||||
|
while (rootScanner.next(key, results)) {
|
||||||
|
HRegionInfo info = Writables.getHRegionInfoOrNull(
|
||||||
|
results.get(HConstants.COL_REGIONINFO));
|
||||||
|
if (info == null) {
|
||||||
|
LOG.warn("region info is null for row " + key.getRow() +
|
||||||
|
" in table " + HConstants.ROOT_TABLE_NAME);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
if (!listener.processRow(info)) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
results.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
rootScanner.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Scans a meta region. For every region found, calls the listener with
|
||||||
|
* the HRegionInfo of the region.
|
||||||
|
*
|
||||||
|
* @param metaRegionInfo HRegionInfo for meta region
|
||||||
|
* @param listener method to be called for each meta region found
|
||||||
|
* @throws IOException
|
||||||
|
*/
|
||||||
|
public void scanMetaRegion(HRegionInfo metaRegionInfo,
|
||||||
|
ScannerListener listener) throws IOException {
|
||||||
|
if (!initialized) {
|
||||||
|
throw new IllegalStateException("Must call initialize method first.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// Open meta region so we can scan it
|
||||||
|
|
||||||
|
HRegion metaRegion = openMetaRegion(metaRegionInfo);
|
||||||
|
|
||||||
|
HScannerInterface metaScanner = metaRegion.getScanner(
|
||||||
|
HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
|
||||||
|
HConstants.LATEST_TIMESTAMP, null);
|
||||||
|
|
||||||
|
try {
|
||||||
|
HStoreKey key = new HStoreKey();
|
||||||
|
SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
||||||
|
while (metaScanner.next(key, results)) {
|
||||||
|
HRegionInfo info = Writables.getHRegionInfoOrNull(
|
||||||
|
results.get(HConstants.COL_REGIONINFO));
|
||||||
|
if (info == null) {
|
||||||
|
LOG.warn("region info is null for row " + key.getRow() +
|
||||||
|
" in table " + HConstants.META_TABLE_NAME);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
if (!listener.processRow(info)) {
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
results.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
metaScanner.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void openRootRegion() throws IOException {
|
||||||
|
this.rootRegion = HRegion.openHRegion(HRegionInfo.rootRegionInfo,
|
||||||
|
this.rootdir, this.log, this.conf);
|
||||||
|
this.rootRegion.compactStores();
|
||||||
|
}
|
||||||
|
|
||||||
|
private HRegion openMetaRegion(HRegionInfo metaInfo) throws IOException {
|
||||||
|
HRegion meta =
|
||||||
|
HRegion.openHRegion(metaInfo, this.rootdir, this.log, this.conf);
|
||||||
|
meta.compactStores();
|
||||||
|
return meta;
|
||||||
|
}
|
||||||
|
}
|
@ -21,7 +21,6 @@
|
|||||||
package org.apache.hadoop.hbase.util;
|
package org.apache.hadoop.hbase.util;
|
||||||
|
|
||||||
import java.io.BufferedReader;
|
import java.io.BufferedReader;
|
||||||
import java.io.FileNotFoundException;
|
|
||||||
import java.io.InputStreamReader;
|
import java.io.InputStreamReader;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
@ -31,8 +30,6 @@ import java.util.Map;
|
|||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.SortedMap;
|
|
||||||
import java.util.TreeMap;
|
|
||||||
|
|
||||||
import org.apache.commons.cli.CommandLine;
|
import org.apache.commons.cli.CommandLine;
|
||||||
import org.apache.commons.cli.Option;
|
import org.apache.commons.cli.Option;
|
||||||
@ -57,11 +54,8 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
|
|||||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||||
import org.apache.hadoop.hbase.HConstants;
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
import org.apache.hadoop.hbase.HRegionInfo;
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
import org.apache.hadoop.hbase.HScannerInterface;
|
|
||||||
import org.apache.hadoop.hbase.HStoreKey;
|
|
||||||
import org.apache.hadoop.hbase.MasterNotRunningException;
|
import org.apache.hadoop.hbase.MasterNotRunningException;
|
||||||
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HLog;
|
|
||||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
import org.apache.hadoop.hbase.regionserver.HStore;
|
import org.apache.hadoop.hbase.regionserver.HStore;
|
||||||
|
|
||||||
@ -75,6 +69,9 @@ public class Migrate extends Configured implements Tool {
|
|||||||
private static final String OLD_PREFIX = "hregion_";
|
private static final String OLD_PREFIX = "hregion_";
|
||||||
|
|
||||||
private final HBaseConfiguration conf;
|
private final HBaseConfiguration conf;
|
||||||
|
FileSystem fs;
|
||||||
|
Path rootdir;
|
||||||
|
MetaUtils utils;
|
||||||
|
|
||||||
/** Action to take when an extra file or unrecoverd log file is found */
|
/** Action to take when an extra file or unrecoverd log file is found */
|
||||||
private static String ACTIONS = "abort|ignore|delete|prompt";
|
private static String ACTIONS = "abort|ignore|delete|prompt";
|
||||||
@ -99,8 +96,6 @@ public class Migrate extends Configured implements Tool {
|
|||||||
options.put("prompt", ACTION.PROMPT);
|
options.put("prompt", ACTION.PROMPT);
|
||||||
}
|
}
|
||||||
|
|
||||||
private FileSystem fs = null;
|
|
||||||
private Path rootdir = null;
|
|
||||||
private boolean readOnly = false;
|
private boolean readOnly = false;
|
||||||
private boolean migrationNeeded = false;
|
private boolean migrationNeeded = false;
|
||||||
private boolean newRootRegion = false;
|
private boolean newRootRegion = false;
|
||||||
@ -121,7 +116,6 @@ public class Migrate extends Configured implements Tool {
|
|||||||
public Migrate(HBaseConfiguration conf) {
|
public Migrate(HBaseConfiguration conf) {
|
||||||
super(conf);
|
super(conf);
|
||||||
this.conf = conf;
|
this.conf = conf;
|
||||||
conf.setInt("hbase.client.retries.number", 1);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@ -131,37 +125,37 @@ public class Migrate extends Configured implements Tool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
// Verify file system is up.
|
||||||
fs = FileSystem.get(conf); // get DFS handle
|
fs = FileSystem.get(conf); // get DFS handle
|
||||||
|
|
||||||
LOG.info("Verifying that file system is available...");
|
LOG.info("Verifying that file system is available...");
|
||||||
if (!FSUtils.isFileSystemAvailable(fs)) {
|
FSUtils.checkFileSystemAvailable(fs);
|
||||||
throw new IOException(
|
} catch (IOException e) {
|
||||||
"Filesystem must be available for upgrade to run.");
|
LOG.fatal("File system is not available", e);
|
||||||
|
return -1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Verify HBase is down
|
||||||
LOG.info("Verifying that HBase is not running...");
|
LOG.info("Verifying that HBase is not running...");
|
||||||
try {
|
try {
|
||||||
HBaseAdmin admin = new HBaseAdmin(conf);
|
HBaseAdmin.checkHBaseAvailable(conf);
|
||||||
if (admin.isMasterRunning()) {
|
LOG.fatal("HBase cluster must be off-line.");
|
||||||
throw new IllegalStateException(
|
return -1;
|
||||||
"HBase cluster must be off-line during upgrade.");
|
|
||||||
}
|
|
||||||
} catch (MasterNotRunningException e) {
|
} catch (MasterNotRunningException e) {
|
||||||
// ignore
|
// Expected. Ignore.
|
||||||
}
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
|
||||||
|
// Initialize MetaUtils and and get the root of the HBase installation
|
||||||
|
|
||||||
|
this.utils = new MetaUtils(conf);
|
||||||
|
this.rootdir = utils.initialize();
|
||||||
|
|
||||||
LOG.info("Starting upgrade" + (readOnly ? " check" : ""));
|
LOG.info("Starting upgrade" + (readOnly ? " check" : ""));
|
||||||
|
|
||||||
rootdir = fs.makeQualified(new Path(this.conf.get(HConstants.HBASE_DIR)));
|
|
||||||
|
|
||||||
if (!fs.exists(rootdir)) {
|
|
||||||
throw new FileNotFoundException("HBase root directory " +
|
|
||||||
rootdir.toString() + " does not exist.");
|
|
||||||
}
|
|
||||||
|
|
||||||
// See if there is a file system version file
|
// See if there is a file system version file
|
||||||
|
|
||||||
String version = FSUtils.checkVersion(fs, rootdir);
|
String version = FSUtils.getVersion(fs, rootdir);
|
||||||
if (version != null &&
|
if (version != null &&
|
||||||
version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0) {
|
version.compareTo(HConstants.FILE_SYSTEM_VERSION) == 0) {
|
||||||
LOG.info("No upgrade necessary.");
|
LOG.info("No upgrade necessary.");
|
||||||
@ -195,6 +189,11 @@ public class Migrate extends Configured implements Tool {
|
|||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.fatal("Upgrade" + (readOnly ? " check" : "") + " failed", e);
|
LOG.fatal("Upgrade" + (readOnly ? " check" : "") + " failed", e);
|
||||||
return -1;
|
return -1;
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
if (utils != null && utils.isInitialized()) {
|
||||||
|
utils.shutdown();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -217,12 +216,11 @@ public class Migrate extends Configured implements Tool {
|
|||||||
if (!newRootRegion) {
|
if (!newRootRegion) {
|
||||||
// find root region
|
// find root region
|
||||||
|
|
||||||
Path rootRegion = new Path(rootdir,
|
String rootRegion = OLD_PREFIX +
|
||||||
OLD_PREFIX + HRegionInfo.rootRegionInfo.getEncodedName());
|
HRegionInfo.rootRegionInfo.getEncodedName();
|
||||||
|
|
||||||
if (!fs.exists(rootRegion)) {
|
if (!fs.exists(new Path(rootdir, rootRegion))) {
|
||||||
throw new IOException("Cannot find root region " +
|
throw new IOException("Cannot find root region " + rootRegion);
|
||||||
rootRegion.toString());
|
|
||||||
} else if (readOnly) {
|
} else if (readOnly) {
|
||||||
migrationNeeded = true;
|
migrationNeeded = true;
|
||||||
} else {
|
} else {
|
||||||
@ -328,8 +326,7 @@ public class Migrate extends Configured implements Tool {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private void migrateRegionDir(Text tableName, Path oldPath)
|
void migrateRegionDir(Text tableName, String oldPath)throws IOException {
|
||||||
throws IOException {
|
|
||||||
|
|
||||||
// Create directory where table will live
|
// Create directory where table will live
|
||||||
|
|
||||||
@ -338,9 +335,9 @@ public class Migrate extends Configured implements Tool {
|
|||||||
|
|
||||||
// Move the old region directory under the table directory
|
// Move the old region directory under the table directory
|
||||||
|
|
||||||
Path newPath =
|
Path newPath = new Path(tableDir,
|
||||||
new Path(tableDir, oldPath.getName().substring(OLD_PREFIX.length()));
|
oldPath.substring(OLD_PREFIX.length()));
|
||||||
fs.rename(oldPath, newPath);
|
fs.rename(new Path(rootdir, oldPath), newPath);
|
||||||
|
|
||||||
processRegionSubDirs(fs, newPath);
|
processRegionSubDirs(fs, newPath);
|
||||||
}
|
}
|
||||||
@ -375,97 +372,33 @@ public class Migrate extends Configured implements Tool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private void scanRootRegion() throws IOException {
|
private void scanRootRegion() throws IOException {
|
||||||
HLog log = new HLog(fs, new Path(rootdir, HConstants.HREGION_LOGDIR_NAME),
|
utils.scanRootRegion(
|
||||||
conf, null);
|
new MetaUtils.ScannerListener() {
|
||||||
|
public boolean processRow(HRegionInfo info) throws IOException {
|
||||||
try {
|
|
||||||
// Open root region so we can scan it
|
|
||||||
|
|
||||||
HRegion rootRegion = new HRegion(
|
|
||||||
new Path(rootdir, HConstants.ROOT_TABLE_NAME.toString()), log, fs, conf,
|
|
||||||
HRegionInfo.rootRegionInfo, null, null);
|
|
||||||
|
|
||||||
try {
|
|
||||||
HScannerInterface rootScanner = rootRegion.getScanner(
|
|
||||||
HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
|
|
||||||
HConstants.LATEST_TIMESTAMP, null);
|
|
||||||
|
|
||||||
try {
|
|
||||||
HStoreKey key = new HStoreKey();
|
|
||||||
SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
|
||||||
while (rootScanner.next(key, results)) {
|
|
||||||
HRegionInfo info = Writables.getHRegionInfoOrNull(
|
|
||||||
results.get(HConstants.COL_REGIONINFO));
|
|
||||||
if (info == null) {
|
|
||||||
LOG.warn("region info is null for row " + key.getRow() +
|
|
||||||
" in table " + HConstants.ROOT_TABLE_NAME);
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
// First move the meta region to where it should be and rename
|
// First move the meta region to where it should be and rename
|
||||||
// subdirectories as necessary
|
// subdirectories as necessary
|
||||||
|
|
||||||
migrateRegionDir(HConstants.META_TABLE_NAME,
|
migrateRegionDir(HConstants.META_TABLE_NAME,
|
||||||
new Path(rootdir, OLD_PREFIX + info.getEncodedName()));
|
OLD_PREFIX + info.getEncodedName());
|
||||||
|
|
||||||
// Now scan and process the meta table
|
|
||||||
|
|
||||||
scanMetaRegion(log, info);
|
|
||||||
}
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
rootScanner.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
rootRegion.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
} finally {
|
|
||||||
log.closeAndDelete();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private void scanMetaRegion(HLog log, HRegionInfo info) throws IOException {
|
|
||||||
|
|
||||||
HRegion metaRegion = new HRegion(
|
|
||||||
new Path(rootdir, info.getTableDesc().getName().toString()), log, fs,
|
|
||||||
conf, info, null, null);
|
|
||||||
|
|
||||||
try {
|
|
||||||
HScannerInterface metaScanner = metaRegion.getScanner(
|
|
||||||
HConstants.COL_REGIONINFO_ARRAY, HConstants.EMPTY_START_ROW,
|
|
||||||
HConstants.LATEST_TIMESTAMP, null);
|
|
||||||
|
|
||||||
try {
|
|
||||||
HStoreKey key = new HStoreKey();
|
|
||||||
SortedMap<Text, byte[]> results = new TreeMap<Text, byte[]>();
|
|
||||||
while (metaScanner.next(key, results)) {
|
|
||||||
HRegionInfo region = Writables.getHRegionInfoOrNull(
|
|
||||||
results.get(HConstants.COL_REGIONINFO));
|
|
||||||
if (region == null) {
|
|
||||||
LOG.warn("region info is null for row " + key.getRow() +
|
|
||||||
" in table " + HConstants.META_TABLE_NAME);
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
utils.scanMetaRegion(info,
|
||||||
|
new MetaUtils.ScannerListener() {
|
||||||
|
public boolean processRow(HRegionInfo tableInfo)
|
||||||
|
throws IOException {
|
||||||
// Move the region to where it should be and rename
|
// Move the region to where it should be and rename
|
||||||
// subdirectories as necessary
|
// subdirectories as necessary
|
||||||
|
|
||||||
migrateRegionDir(region.getTableDesc().getName(),
|
migrateRegionDir(tableInfo.getTableDesc().getName(),
|
||||||
new Path(rootdir, OLD_PREFIX + region.getEncodedName()));
|
OLD_PREFIX + tableInfo.getEncodedName());
|
||||||
|
return true;
|
||||||
results.clear();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
} finally {
|
|
||||||
metaScanner.close();
|
|
||||||
}
|
}
|
||||||
|
);
|
||||||
} finally {
|
return true;
|
||||||
metaRegion.close();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
private void extraRegions() throws IOException {
|
private void extraRegions() throws IOException {
|
||||||
FileStatus[] stats = fs.listStatus(rootdir);
|
FileStatus[] stats = fs.listStatus(rootdir);
|
||||||
|
@ -600,7 +600,7 @@ implements RegionUnavailableListener {
|
|||||||
Path oldRegion1 = subregions[0].getRegionDir();
|
Path oldRegion1 = subregions[0].getRegionDir();
|
||||||
Path oldRegion2 = subregions[1].getRegionDir();
|
Path oldRegion2 = subregions[1].getRegionDir();
|
||||||
startTime = System.currentTimeMillis();
|
startTime = System.currentTimeMillis();
|
||||||
r = HRegion.closeAndMerge(subregions[0], subregions[1]);
|
r = HRegion.mergeAdjacent(subregions[0], subregions[1]);
|
||||||
region = new HRegionIncommon(r);
|
region = new HRegionIncommon(r);
|
||||||
System.out.println("Merge regions elapsed time: "
|
System.out.println("Merge regions elapsed time: "
|
||||||
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
+ ((System.currentTimeMillis() - startTime) / 1000.0));
|
||||||
|
310
src/test/org/apache/hadoop/hbase/util/TestMergeTool.java
Normal file
310
src/test/org/apache/hadoop/hbase/util/TestMergeTool.java
Normal file
@ -0,0 +1,310 @@
|
|||||||
|
/**
|
||||||
|
* Copyright 2008 The Apache Software Foundation
|
||||||
|
*
|
||||||
|
* 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.util;
|
||||||
|
|
||||||
|
import org.apache.commons.logging.Log;
|
||||||
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
import org.apache.hadoop.io.Text;
|
||||||
|
import org.apache.hadoop.util.ToolRunner;
|
||||||
|
|
||||||
|
import org.apache.hadoop.dfs.MiniDFSCluster;
|
||||||
|
import org.apache.hadoop.hbase.HBaseTestCase;
|
||||||
|
import org.apache.hadoop.hbase.HColumnDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.HConstants;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HLog;
|
||||||
|
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||||
|
import org.apache.hadoop.hbase.HRegionInfo;
|
||||||
|
import org.apache.hadoop.hbase.HTableDescriptor;
|
||||||
|
import org.apache.hadoop.hbase.StaticTestEnvironment;
|
||||||
|
import org.apache.hadoop.hbase.io.BatchUpdate;
|
||||||
|
import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
|
||||||
|
|
||||||
|
/** Test stand alone merge tool that can merge arbitrary regions */
|
||||||
|
public class TestMergeTool extends HBaseTestCase {
|
||||||
|
static final Log LOG = LogFactory.getLog(TestMergeTool.class);
|
||||||
|
protected static final Text COLUMN_NAME = new Text("contents:");
|
||||||
|
private final HRegionInfo[] sourceRegions = new HRegionInfo[5];
|
||||||
|
private final HRegion[] regions = new HRegion[5];
|
||||||
|
private HTableDescriptor desc;
|
||||||
|
private Text[][] rows;
|
||||||
|
private Path rootdir = null;
|
||||||
|
private MiniDFSCluster dfsCluster = null;
|
||||||
|
private FileSystem fs;
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public void setUp() throws Exception {
|
||||||
|
|
||||||
|
// Create table description
|
||||||
|
|
||||||
|
this.desc = new HTableDescriptor("TestMergeTool");
|
||||||
|
this.desc.addFamily(new HColumnDescriptor(COLUMN_NAME.toString()));
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Create the HRegionInfos for the regions.
|
||||||
|
*/
|
||||||
|
|
||||||
|
// Region 0 will contain the key range [row_0200,row_0300)
|
||||||
|
sourceRegions[0] =
|
||||||
|
new HRegionInfo(this.desc, new Text("row_0200"), new Text("row_0300"));
|
||||||
|
|
||||||
|
// Region 1 will contain the key range [row_0250,row_0400) and overlaps
|
||||||
|
// with Region 0
|
||||||
|
sourceRegions[1] =
|
||||||
|
new HRegionInfo(this.desc, new Text("row_0250"), new Text("row_0400"));
|
||||||
|
|
||||||
|
// Region 2 will contain the key range [row_0100,row_0200) and is adjacent
|
||||||
|
// to Region 0 or the region resulting from the merge of Regions 0 and 1
|
||||||
|
sourceRegions[2] =
|
||||||
|
new HRegionInfo(this.desc, new Text("row_0100"), new Text("row_0200"));
|
||||||
|
|
||||||
|
// Region 3 will contain the key range [row_0500,row_0600) and is not
|
||||||
|
// adjacent to any of Regions 0, 1, 2 or the merged result of any or all
|
||||||
|
// of those regions
|
||||||
|
sourceRegions[3] =
|
||||||
|
new HRegionInfo(this.desc, new Text("row_0500"), new Text("row_0600"));
|
||||||
|
|
||||||
|
// Region 4 will have empty start and end keys and overlaps all regions.
|
||||||
|
sourceRegions[4] =
|
||||||
|
new HRegionInfo(this.desc, HConstants.EMPTY_TEXT, HConstants.EMPTY_TEXT);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Now create some row keys
|
||||||
|
*/
|
||||||
|
this.rows = new Text[5][];
|
||||||
|
this.rows[0] = new Text[] { new Text("row_0210"), new Text("row_0280") };
|
||||||
|
this.rows[1] = new Text[] { new Text("row_0260"), new Text("row_0350") };
|
||||||
|
this.rows[2] = new Text[] { new Text("row_0110"), new Text("row_0175") };
|
||||||
|
this.rows[3] = new Text[] { new Text("row_0525"), new Text("row_0560") };
|
||||||
|
this.rows[4] = new Text[] { new Text("row_0050"), new Text("row_1000") };
|
||||||
|
|
||||||
|
// Start up dfs
|
||||||
|
this.dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||||
|
this.fs = this.dfsCluster.getFileSystem();
|
||||||
|
// Set the hbase.rootdir to be the home directory in mini dfs.
|
||||||
|
this.rootdir = new Path(this.fs.getHomeDirectory(), "hbase");
|
||||||
|
this.conf.set(HConstants.HBASE_DIR, this.rootdir.toString());
|
||||||
|
|
||||||
|
// Note: we must call super.setUp after starting the mini cluster or
|
||||||
|
// we will end up with a local file system
|
||||||
|
|
||||||
|
super.setUp();
|
||||||
|
|
||||||
|
try {
|
||||||
|
/*
|
||||||
|
* Create the regions we will merge
|
||||||
|
*/
|
||||||
|
for (int i = 0; i < sourceRegions.length; i++) {
|
||||||
|
regions[i] =
|
||||||
|
HRegion.createHRegion(this.sourceRegions[i], this.rootdir, this.conf);
|
||||||
|
/*
|
||||||
|
* Insert data
|
||||||
|
*/
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
Text row = rows[i][j];
|
||||||
|
BatchUpdate b = new BatchUpdate(row);
|
||||||
|
b.put(COLUMN_NAME,
|
||||||
|
new ImmutableBytesWritable(
|
||||||
|
row.getBytes(), 0, row.getLength()
|
||||||
|
).get()
|
||||||
|
);
|
||||||
|
regions[i].batchUpdate(b);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// Create root region
|
||||||
|
HRegion root = HRegion.createHRegion(HRegionInfo.rootRegionInfo,
|
||||||
|
this.rootdir, this.conf);
|
||||||
|
// Create meta region
|
||||||
|
HRegion meta = HRegion.createHRegion(HRegionInfo.firstMetaRegionInfo,
|
||||||
|
this.rootdir, this.conf);
|
||||||
|
// Insert meta into root region
|
||||||
|
HRegion.addRegionToMETA(root, meta);
|
||||||
|
// Insert the regions we created into the meta
|
||||||
|
for(int i = 0; i < regions.length; i++) {
|
||||||
|
HRegion.addRegionToMETA(meta, regions[i]);
|
||||||
|
}
|
||||||
|
// Close root and meta regions
|
||||||
|
root.close();
|
||||||
|
root.getLog().closeAndDelete();
|
||||||
|
meta.close();
|
||||||
|
meta.getLog().closeAndDelete();
|
||||||
|
|
||||||
|
} catch (Exception e) {
|
||||||
|
StaticTestEnvironment.shutdownDfs(dfsCluster);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/** {@inheritDoc} */
|
||||||
|
@Override
|
||||||
|
public void tearDown() throws Exception {
|
||||||
|
super.tearDown();
|
||||||
|
StaticTestEnvironment.shutdownDfs(dfsCluster);
|
||||||
|
}
|
||||||
|
|
||||||
|
/** @throws Exception */
|
||||||
|
public void testMergeTool() throws Exception {
|
||||||
|
// First verify we can read the rows from the source regions and that they
|
||||||
|
// contain the right data.
|
||||||
|
for (int i = 0; i < regions.length; i++) {
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
byte[] bytes = regions[i].get(rows[i][j], COLUMN_NAME).getValue();
|
||||||
|
assertNotNull(bytes);
|
||||||
|
Text value = new Text(bytes);
|
||||||
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
}
|
||||||
|
// Close the region and delete the log
|
||||||
|
regions[i].close();
|
||||||
|
regions[i].getLog().closeAndDelete();
|
||||||
|
}
|
||||||
|
|
||||||
|
// Create a log that we can reuse when we need to open regions
|
||||||
|
|
||||||
|
HLog log = new HLog(this.fs,
|
||||||
|
new Path("/tmp", HConstants.HREGION_LOGDIR_NAME + "_" +
|
||||||
|
System.currentTimeMillis()
|
||||||
|
),
|
||||||
|
this.conf, null
|
||||||
|
);
|
||||||
|
try {
|
||||||
|
/*
|
||||||
|
* Merge Region 0 and Region 1
|
||||||
|
*/
|
||||||
|
LOG.info("merging regions 0 and 1");
|
||||||
|
Merge merger = new Merge(this.conf);
|
||||||
|
ToolRunner.run(merger,
|
||||||
|
new String[] {
|
||||||
|
this.desc.getName().toString(),
|
||||||
|
this.sourceRegions[0].getRegionName().toString(),
|
||||||
|
this.sourceRegions[1].getRegionName().toString()
|
||||||
|
}
|
||||||
|
);
|
||||||
|
HRegionInfo mergedInfo = merger.getMergedHRegionInfo();
|
||||||
|
|
||||||
|
// Now verify that we can read all the rows from regions 0, 1
|
||||||
|
// in the new merged region.
|
||||||
|
HRegion merged =
|
||||||
|
HRegion.openHRegion(mergedInfo, this.rootdir, log, this.conf);
|
||||||
|
|
||||||
|
for (int i = 0; i < 2 ; i++) {
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue();
|
||||||
|
assertNotNull(rows[i][j].toString(), bytes);
|
||||||
|
Text value = new Text(bytes);
|
||||||
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
merged.close();
|
||||||
|
LOG.info("verified merge of regions 0 and 1");
|
||||||
|
/*
|
||||||
|
* Merge the result of merging regions 0 and 1 with region 2
|
||||||
|
*/
|
||||||
|
LOG.info("merging regions 0+1 and 2");
|
||||||
|
merger = new Merge(this.conf);
|
||||||
|
ToolRunner.run(merger,
|
||||||
|
new String[] {
|
||||||
|
this.desc.getName().toString(),
|
||||||
|
mergedInfo.getRegionName().toString(),
|
||||||
|
this.sourceRegions[2].getRegionName().toString()
|
||||||
|
}
|
||||||
|
);
|
||||||
|
mergedInfo = merger.getMergedHRegionInfo();
|
||||||
|
|
||||||
|
// Now verify that we can read all the rows from regions 0, 1 and 2
|
||||||
|
// in the new merged region.
|
||||||
|
|
||||||
|
merged = HRegion.openHRegion(mergedInfo, this.rootdir, log, this.conf);
|
||||||
|
|
||||||
|
for (int i = 0; i < 3 ; i++) {
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue();
|
||||||
|
assertNotNull(bytes);
|
||||||
|
Text value = new Text(bytes);
|
||||||
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
merged.close();
|
||||||
|
LOG.info("verified merge of regions 0+1 and 2");
|
||||||
|
/*
|
||||||
|
* Merge the result of merging regions 0, 1 and 2 with region 3
|
||||||
|
*/
|
||||||
|
LOG.info("merging regions 0+1+2 and 3");
|
||||||
|
merger = new Merge(this.conf);
|
||||||
|
ToolRunner.run(merger,
|
||||||
|
new String[] {
|
||||||
|
this.desc.getName().toString(),
|
||||||
|
mergedInfo.getRegionName().toString(),
|
||||||
|
this.sourceRegions[3].getRegionName().toString()
|
||||||
|
}
|
||||||
|
);
|
||||||
|
mergedInfo = merger.getMergedHRegionInfo();
|
||||||
|
|
||||||
|
// Now verify that we can read all the rows from regions 0, 1, 2 and 3
|
||||||
|
// in the new merged region.
|
||||||
|
|
||||||
|
merged = HRegion.openHRegion(mergedInfo, this.rootdir, log, this.conf);
|
||||||
|
|
||||||
|
for (int i = 0; i < 4 ; i++) {
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue();
|
||||||
|
assertNotNull(bytes);
|
||||||
|
Text value = new Text(bytes);
|
||||||
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
merged.close();
|
||||||
|
LOG.info("verified merge of regions 0+1+2 and 3");
|
||||||
|
/*
|
||||||
|
* Merge the result of merging regions 0, 1, 2 and 3 with region 4
|
||||||
|
*/
|
||||||
|
LOG.info("merging regions 0+1+2+3 and 4");
|
||||||
|
merger = new Merge(this.conf);
|
||||||
|
ToolRunner.run(merger,
|
||||||
|
new String[] {
|
||||||
|
this.desc.getName().toString(),
|
||||||
|
mergedInfo.getRegionName().toString(),
|
||||||
|
this.sourceRegions[4].getRegionName().toString()
|
||||||
|
}
|
||||||
|
);
|
||||||
|
mergedInfo = merger.getMergedHRegionInfo();
|
||||||
|
|
||||||
|
// Now verify that we can read all the rows from the new merged region.
|
||||||
|
|
||||||
|
merged = HRegion.openHRegion(mergedInfo, this.rootdir, log, this.conf);
|
||||||
|
|
||||||
|
for (int i = 0; i < rows.length ; i++) {
|
||||||
|
for (int j = 0; j < rows[i].length; j++) {
|
||||||
|
byte[] bytes = merged.get(rows[i][j], COLUMN_NAME).getValue();
|
||||||
|
assertNotNull(bytes);
|
||||||
|
Text value = new Text(bytes);
|
||||||
|
assertTrue(value.equals(rows[i][j]));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
merged.close();
|
||||||
|
LOG.info("verified merge of regions 0+1+2+3 and 4");
|
||||||
|
|
||||||
|
} finally {
|
||||||
|
log.closeAndDelete();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
@ -76,8 +76,8 @@ public class TestMigrate extends HBaseTestCase {
|
|||||||
try {
|
try {
|
||||||
dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
dfsCluster = new MiniDFSCluster(conf, 2, true, (String[])null);
|
||||||
// Set the hbase.rootdir to be the home directory in mini dfs.
|
// Set the hbase.rootdir to be the home directory in mini dfs.
|
||||||
this.conf.set(HConstants.HBASE_DIR,
|
this.conf.set(HConstants.HBASE_DIR, new Path(
|
||||||
dfsCluster.getFileSystem().getHomeDirectory().toString());
|
dfsCluster.getFileSystem().getHomeDirectory(), "hbase").toString());
|
||||||
FileSystem dfs = dfsCluster.getFileSystem();
|
FileSystem dfs = dfsCluster.getFileSystem();
|
||||||
Path root = dfs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
|
Path root = dfs.makeQualified(new Path(conf.get(HConstants.HBASE_DIR)));
|
||||||
dfs.mkdirs(root);
|
dfs.mkdirs(root);
|
||||||
@ -177,13 +177,12 @@ public class TestMigrate extends HBaseTestCase {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
for (int i = 0; i < stats.length; i++) {
|
for (int i = 0; i < stats.length; i++) {
|
||||||
String relativePath =
|
String path = stats[i].getPath().toString();
|
||||||
stats[i].getPath().toString().substring(rootdirlength);
|
|
||||||
if (stats[i].isDir()) {
|
if (stats[i].isDir()) {
|
||||||
System.out.println("d " + relativePath);
|
System.out.println("d " + path);
|
||||||
listPaths(fs, stats[i].getPath(), rootdirlength);
|
listPaths(fs, stats[i].getPath(), rootdirlength);
|
||||||
} else {
|
} else {
|
||||||
System.out.println("f " + relativePath + " size=" + stats[i].getLen());
|
System.out.println("f " + path + " size=" + stats[i].getLen());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user