|
@@ -61,21 +61,21 @@ 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.
|
|
|
|
|
|
- if (srcA.getStartKey() == null) {
|
|
|
|
- if (srcB.getStartKey() == null) {
|
|
|
|
|
|
+ if(srcA.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");
|
|
}
|
|
}
|
|
// A's start key is null but B's isn't. Assume A comes before B
|
|
// A's start key is null but B's isn't. Assume A comes before B
|
|
|
|
|
|
- } else if ((srcB.getStartKey() == null) // A is not null but B is
|
|
|
|
- || (srcA.getStartKey().compareTo(srcB.getStartKey()) > 0)) { // A > B
|
|
|
|
|
|
+ } else if((srcB.getStartKey() == null) // A is not null but B is
|
|
|
|
+ || (srcA.getStartKey().compareTo(srcB.getStartKey()) > 0)) { // A > B
|
|
|
|
|
|
HRegion tmp = srcA;
|
|
HRegion tmp = srcA;
|
|
srcA = srcB;
|
|
srcA = srcB;
|
|
srcB = tmp;
|
|
srcB = tmp;
|
|
}
|
|
}
|
|
|
|
|
|
- if (!srcA.getEndKey().equals(srcB.getStartKey())) {
|
|
|
|
|
|
+ if (! srcA.getEndKey().equals(srcB.getStartKey())) {
|
|
throw new IOException("Cannot merge non-adjacent regions");
|
|
throw new IOException("Cannot merge non-adjacent regions");
|
|
}
|
|
}
|
|
|
|
|
|
@@ -89,7 +89,7 @@ public class HRegion implements HConstants {
|
|
Text endKey = srcB.getEndKey();
|
|
Text endKey = srcB.getEndKey();
|
|
|
|
|
|
Path merges = new Path(srcA.getRegionDir(), MERGEDIR);
|
|
Path merges = new Path(srcA.getRegionDir(), MERGEDIR);
|
|
- if (!fs.exists(merges)) {
|
|
|
|
|
|
+ if(! fs.exists(merges)) {
|
|
fs.mkdirs(merges);
|
|
fs.mkdirs(merges);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -98,14 +98,14 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
Path newRegionDir = HStoreFile.getHRegionDir(merges, newRegionInfo.regionName);
|
|
Path newRegionDir = HStoreFile.getHRegionDir(merges, newRegionInfo.regionName);
|
|
|
|
|
|
- if (fs.exists(newRegionDir)) {
|
|
|
|
|
|
+ if(fs.exists(newRegionDir)) {
|
|
throw new IOException("Cannot merge; target file collision at " + newRegionDir);
|
|
throw new IOException("Cannot merge; target file collision at " + newRegionDir);
|
|
}
|
|
}
|
|
|
|
|
|
LOG.info("starting merge of regions: " + srcA.getRegionName() + " and "
|
|
LOG.info("starting merge of regions: " + srcA.getRegionName() + " and "
|
|
- + srcB.getRegionName() + " new region start key is '"
|
|
|
|
- + (startKey == null ? "" : startKey) + "', end key is '"
|
|
|
|
- + (endKey == null ? "" : endKey) + "'");
|
|
|
|
|
|
+ + srcB.getRegionName() + " new region start key is '"
|
|
|
|
+ + (startKey == null ? "" : startKey) + "', end key is '"
|
|
|
|
+ + (endKey == null ? "" : endKey) + "'");
|
|
|
|
|
|
// Flush each of the sources, and merge their files into a single
|
|
// Flush each of the sources, and merge their files into a single
|
|
// target for each column family.
|
|
// target for each column family.
|
|
@@ -114,10 +114,10 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
TreeSet<HStoreFile> alreadyMerged = new TreeSet<HStoreFile>();
|
|
TreeSet<HStoreFile> alreadyMerged = new TreeSet<HStoreFile>();
|
|
TreeMap<Text, Vector<HStoreFile>> filesToMerge = new TreeMap<Text, Vector<HStoreFile>>();
|
|
TreeMap<Text, Vector<HStoreFile>> filesToMerge = new TreeMap<Text, Vector<HStoreFile>>();
|
|
- for(Iterator<HStoreFile> it = srcA.flushcache(true).iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = srcA.flushcache(true).iterator(); it.hasNext(); ) {
|
|
HStoreFile src = it.next();
|
|
HStoreFile src = it.next();
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
- if (v == null) {
|
|
|
|
|
|
+ if(v == null) {
|
|
v = new Vector<HStoreFile>();
|
|
v = new Vector<HStoreFile>();
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
}
|
|
}
|
|
@@ -126,10 +126,10 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
LOG.debug("flushing and getting file names for region " + srcB.getRegionName());
|
|
LOG.debug("flushing and getting file names for region " + srcB.getRegionName());
|
|
|
|
|
|
- for(Iterator<HStoreFile> it = srcB.flushcache(true).iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = srcB.flushcache(true).iterator(); it.hasNext(); ) {
|
|
HStoreFile src = it.next();
|
|
HStoreFile src = it.next();
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
- if (v == null) {
|
|
|
|
|
|
+ if(v == null) {
|
|
v = new Vector<HStoreFile>();
|
|
v = new Vector<HStoreFile>();
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
}
|
|
}
|
|
@@ -138,11 +138,11 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
LOG.debug("merging stores");
|
|
LOG.debug("merging stores");
|
|
|
|
|
|
- for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
|
Text colFamily = it.next();
|
|
Text colFamily = it.next();
|
|
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
|
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
|
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
|
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
|
- colFamily, Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ colFamily, Math.abs(rand.nextLong()));
|
|
|
|
|
|
dst.mergeStoreFiles(srcFiles, fs, conf);
|
|
dst.mergeStoreFiles(srcFiles, fs, conf);
|
|
alreadyMerged.addAll(srcFiles);
|
|
alreadyMerged.addAll(srcFiles);
|
|
@@ -153,15 +153,15 @@ public class HRegion implements HConstants {
|
|
// of any last-minute inserts
|
|
// of any last-minute inserts
|
|
|
|
|
|
LOG.debug("flushing changes since start of merge for region "
|
|
LOG.debug("flushing changes since start of merge for region "
|
|
- + srcA.getRegionName());
|
|
|
|
|
|
+ + srcA.getRegionName());
|
|
|
|
|
|
filesToMerge.clear();
|
|
filesToMerge.clear();
|
|
- for(Iterator<HStoreFile> it = srcA.close().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = srcA.close().iterator(); it.hasNext(); ) {
|
|
HStoreFile src = it.next();
|
|
HStoreFile src = it.next();
|
|
|
|
|
|
- if (!alreadyMerged.contains(src)) {
|
|
|
|
|
|
+ if(! alreadyMerged.contains(src)) {
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
- if (v == null) {
|
|
|
|
|
|
+ if(v == null) {
|
|
v = new Vector<HStoreFile>();
|
|
v = new Vector<HStoreFile>();
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
}
|
|
}
|
|
@@ -170,14 +170,14 @@ public class HRegion implements HConstants {
|
|
}
|
|
}
|
|
|
|
|
|
LOG.debug("flushing changes since start of merge for region "
|
|
LOG.debug("flushing changes since start of merge for region "
|
|
- + srcB.getRegionName());
|
|
|
|
|
|
+ + srcB.getRegionName());
|
|
|
|
|
|
- for(Iterator<HStoreFile> it = srcB.close().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = srcB.close().iterator(); it.hasNext(); ) {
|
|
HStoreFile src = it.next();
|
|
HStoreFile src = it.next();
|
|
|
|
|
|
- if (!alreadyMerged.contains(src)) {
|
|
|
|
|
|
+ if(! alreadyMerged.contains(src)) {
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
Vector<HStoreFile> v = filesToMerge.get(src.getColFamily());
|
|
- if (v == null) {
|
|
|
|
|
|
+ if(v == null) {
|
|
v = new Vector<HStoreFile>();
|
|
v = new Vector<HStoreFile>();
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
filesToMerge.put(src.getColFamily(), v);
|
|
}
|
|
}
|
|
@@ -187,11 +187,11 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
LOG.debug("merging changes since start of merge");
|
|
LOG.debug("merging changes since start of merge");
|
|
|
|
|
|
- for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<Text> it = filesToMerge.keySet().iterator(); it.hasNext(); ) {
|
|
Text colFamily = it.next();
|
|
Text colFamily = it.next();
|
|
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
|
Vector<HStoreFile> srcFiles = filesToMerge.get(colFamily);
|
|
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
|
HStoreFile dst = new HStoreFile(conf, merges, newRegionInfo.regionName,
|
|
- colFamily, Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ colFamily, Math.abs(rand.nextLong()));
|
|
|
|
|
|
dst.mergeStoreFiles(srcFiles, fs, conf);
|
|
dst.mergeStoreFiles(srcFiles, fs, conf);
|
|
}
|
|
}
|
|
@@ -199,7 +199,7 @@ public class HRegion implements HConstants {
|
|
// Done
|
|
// Done
|
|
|
|
|
|
HRegion dstRegion = new HRegion(dir, log, fs, conf, newRegionInfo,
|
|
HRegion dstRegion = new HRegion(dir, log, fs, conf, newRegionInfo,
|
|
- newRegionDir, null);
|
|
|
|
|
|
+ newRegionDir, null);
|
|
|
|
|
|
// Get rid of merges directory
|
|
// Get rid of merges directory
|
|
|
|
|
|
@@ -284,7 +284,7 @@ public class HRegion implements HConstants {
|
|
* written-to before), then read it from the supplied path.
|
|
* written-to before), then read it from the supplied path.
|
|
*/
|
|
*/
|
|
public HRegion(Path dir, HLog log, FileSystem fs, Configuration conf,
|
|
public HRegion(Path dir, HLog log, FileSystem fs, Configuration conf,
|
|
- HRegionInfo regionInfo, Path initialFiles, Path oldLogFile) throws IOException {
|
|
|
|
|
|
+ HRegionInfo regionInfo, Path initialFiles, Path oldLogFile) throws IOException {
|
|
|
|
|
|
this.dir = dir;
|
|
this.dir = dir;
|
|
this.log = log;
|
|
this.log = log;
|
|
@@ -303,29 +303,29 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
// Move prefab HStore files into place (if any)
|
|
// Move prefab HStore files into place (if any)
|
|
|
|
|
|
- if (initialFiles != null && fs.exists(initialFiles)) {
|
|
|
|
|
|
+ if(initialFiles != null && fs.exists(initialFiles)) {
|
|
fs.rename(initialFiles, regiondir);
|
|
fs.rename(initialFiles, regiondir);
|
|
}
|
|
}
|
|
|
|
|
|
// Load in all the HStores.
|
|
// Load in all the HStores.
|
|
|
|
|
|
for(Iterator<Text> it = this.regionInfo.tableDesc.families().iterator();
|
|
for(Iterator<Text> it = this.regionInfo.tableDesc.families().iterator();
|
|
- it.hasNext();) {
|
|
|
|
|
|
+ it.hasNext(); ) {
|
|
|
|
|
|
- Text colFamily = it.next();
|
|
|
|
|
|
+ Text colFamily = HStoreKey.extractFamily(it.next());
|
|
stores.put(colFamily, new HStore(dir, this.regionInfo.regionName, colFamily,
|
|
stores.put(colFamily, new HStore(dir, this.regionInfo.regionName, colFamily,
|
|
- this.regionInfo.tableDesc.getMaxVersions(), fs, oldLogFile, conf));
|
|
|
|
|
|
+ this.regionInfo.tableDesc.getMaxVersions(), fs, oldLogFile, conf));
|
|
}
|
|
}
|
|
|
|
|
|
// Get rid of any splits or merges that were lost in-progress
|
|
// Get rid of any splits or merges that were lost in-progress
|
|
|
|
|
|
Path splits = new Path(regiondir, SPLITDIR);
|
|
Path splits = new Path(regiondir, SPLITDIR);
|
|
- if (fs.exists(splits)) {
|
|
|
|
|
|
+ if(fs.exists(splits)) {
|
|
fs.delete(splits);
|
|
fs.delete(splits);
|
|
}
|
|
}
|
|
|
|
|
|
Path merges = new Path(regiondir, MERGEDIR);
|
|
Path merges = new Path(regiondir, MERGEDIR);
|
|
- if (fs.exists(merges)) {
|
|
|
|
|
|
+ if(fs.exists(merges)) {
|
|
fs.delete(merges);
|
|
fs.delete(merges);
|
|
}
|
|
}
|
|
|
|
|
|
@@ -345,6 +345,7 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
/** Closes and deletes this HRegion. Called when doing a table deletion, for example */
|
|
/** Closes and deletes this HRegion. Called when doing a table deletion, for example */
|
|
public void closeAndDelete() throws IOException {
|
|
public void closeAndDelete() throws IOException {
|
|
|
|
+ LOG.info("deleting region: " + regionInfo.regionName);
|
|
close();
|
|
close();
|
|
fs.delete(regiondir);
|
|
fs.delete(regiondir);
|
|
}
|
|
}
|
|
@@ -362,7 +363,7 @@ public class HRegion implements HConstants {
|
|
public Vector<HStoreFile> close() throws IOException {
|
|
public Vector<HStoreFile> close() throws IOException {
|
|
boolean shouldClose = false;
|
|
boolean shouldClose = false;
|
|
synchronized(writestate) {
|
|
synchronized(writestate) {
|
|
- if (writestate.closed) {
|
|
|
|
|
|
+ if(writestate.closed) {
|
|
LOG.info("region " + this.regionInfo.regionName + " closed");
|
|
LOG.info("region " + this.regionInfo.regionName + " closed");
|
|
return new Vector<HStoreFile>();
|
|
return new Vector<HStoreFile>();
|
|
}
|
|
}
|
|
@@ -376,13 +377,13 @@ public class HRegion implements HConstants {
|
|
shouldClose = true;
|
|
shouldClose = true;
|
|
}
|
|
}
|
|
|
|
|
|
- if (!shouldClose) {
|
|
|
|
|
|
+ if(! shouldClose) {
|
|
return null;
|
|
return null;
|
|
|
|
|
|
} else {
|
|
} else {
|
|
LOG.info("closing region " + this.regionInfo.regionName);
|
|
LOG.info("closing region " + this.regionInfo.regionName);
|
|
Vector<HStoreFile> allHStoreFiles = internalFlushcache();
|
|
Vector<HStoreFile> allHStoreFiles = internalFlushcache();
|
|
- for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
|
HStore store = it.next();
|
|
HStore store = it.next();
|
|
store.close();
|
|
store.close();
|
|
}
|
|
}
|
|
@@ -406,8 +407,8 @@ public class HRegion implements HConstants {
|
|
* Returns two brand-new (and open) HRegions
|
|
* Returns two brand-new (and open) HRegions
|
|
*/
|
|
*/
|
|
public HRegion[] closeAndSplit(Text midKey) throws IOException {
|
|
public HRegion[] closeAndSplit(Text midKey) throws IOException {
|
|
- if (((regionInfo.startKey.getLength() != 0)
|
|
|
|
- && (regionInfo.startKey.compareTo(midKey) > 0))
|
|
|
|
|
|
+ if(((regionInfo.startKey.getLength() != 0)
|
|
|
|
+ && (regionInfo.startKey.compareTo(midKey) > 0))
|
|
|| ((regionInfo.endKey.getLength() != 0)
|
|
|| ((regionInfo.endKey.getLength() != 0)
|
|
&& (regionInfo.endKey.compareTo(midKey) < 0))) {
|
|
&& (regionInfo.endKey.compareTo(midKey) < 0))) {
|
|
throw new IOException("Region splitkey must lie within region boundaries.");
|
|
throw new IOException("Region splitkey must lie within region boundaries.");
|
|
@@ -419,13 +420,13 @@ public class HRegion implements HConstants {
|
|
// or compactions until close() is called.
|
|
// or compactions until close() is called.
|
|
|
|
|
|
Path splits = new Path(regiondir, SPLITDIR);
|
|
Path splits = new Path(regiondir, SPLITDIR);
|
|
- if (!fs.exists(splits)) {
|
|
|
|
|
|
+ if(! fs.exists(splits)) {
|
|
fs.mkdirs(splits);
|
|
fs.mkdirs(splits);
|
|
}
|
|
}
|
|
|
|
|
|
long regionAId = Math.abs(rand.nextLong());
|
|
long regionAId = Math.abs(rand.nextLong());
|
|
HRegionInfo regionAInfo = new HRegionInfo(regionAId, regionInfo.tableDesc,
|
|
HRegionInfo regionAInfo = new HRegionInfo(regionAId, regionInfo.tableDesc,
|
|
- regionInfo.startKey, midKey);
|
|
|
|
|
|
+ regionInfo.startKey, midKey);
|
|
|
|
|
|
long regionBId = Math.abs(rand.nextLong());
|
|
long regionBId = Math.abs(rand.nextLong());
|
|
HRegionInfo regionBInfo
|
|
HRegionInfo regionBInfo
|
|
@@ -434,24 +435,24 @@ public class HRegion implements HConstants {
|
|
Path dirA = HStoreFile.getHRegionDir(splits, regionAInfo.regionName);
|
|
Path dirA = HStoreFile.getHRegionDir(splits, regionAInfo.regionName);
|
|
Path dirB = HStoreFile.getHRegionDir(splits, regionBInfo.regionName);
|
|
Path dirB = HStoreFile.getHRegionDir(splits, regionBInfo.regionName);
|
|
|
|
|
|
- if (fs.exists(dirA) || fs.exists(dirB)) {
|
|
|
|
|
|
+ if(fs.exists(dirA) || fs.exists(dirB)) {
|
|
throw new IOException("Cannot split; target file collision at " + dirA
|
|
throw new IOException("Cannot split; target file collision at " + dirA
|
|
- + " or " + dirB);
|
|
|
|
|
|
+ + " or " + dirB);
|
|
}
|
|
}
|
|
|
|
|
|
TreeSet<HStoreFile> alreadySplit = new TreeSet<HStoreFile>();
|
|
TreeSet<HStoreFile> alreadySplit = new TreeSet<HStoreFile>();
|
|
Vector<HStoreFile> hstoreFilesToSplit = flushcache(true);
|
|
Vector<HStoreFile> hstoreFilesToSplit = flushcache(true);
|
|
- for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext(); ) {
|
|
HStoreFile hsf = it.next();
|
|
HStoreFile hsf = it.next();
|
|
|
|
|
|
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
|
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
|
- + "/" + hsf.fileId());
|
|
|
|
|
|
+ + "/" + hsf.fileId());
|
|
|
|
|
|
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
|
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
|
- hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
|
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
|
- hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
|
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
|
alreadySplit.add(hsf);
|
|
alreadySplit.add(hsf);
|
|
@@ -461,18 +462,18 @@ public class HRegion implements HConstants {
|
|
// and copy the small remainder
|
|
// and copy the small remainder
|
|
|
|
|
|
hstoreFilesToSplit = close();
|
|
hstoreFilesToSplit = close();
|
|
- for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStoreFile> it = hstoreFilesToSplit.iterator(); it.hasNext(); ) {
|
|
HStoreFile hsf = it.next();
|
|
HStoreFile hsf = it.next();
|
|
|
|
|
|
- if (!alreadySplit.contains(hsf)) {
|
|
|
|
|
|
+ if(! alreadySplit.contains(hsf)) {
|
|
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
|
LOG.debug("splitting HStore " + hsf.getRegionName() + "/" + hsf.getColFamily()
|
|
- + "/" + hsf.fileId());
|
|
|
|
|
|
+ + "/" + hsf.fileId());
|
|
|
|
|
|
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
|
HStoreFile dstA = new HStoreFile(conf, splits, regionAInfo.regionName,
|
|
- hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
|
HStoreFile dstB = new HStoreFile(conf, splits, regionBInfo.regionName,
|
|
- hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
+ hsf.getColFamily(), Math.abs(rand.nextLong()));
|
|
|
|
|
|
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
|
hsf.splitStoreFile(midKey, dstA, dstB, fs, conf);
|
|
}
|
|
}
|
|
@@ -494,7 +495,7 @@ public class HRegion implements HConstants {
|
|
regions[1] = regionB;
|
|
regions[1] = regionB;
|
|
|
|
|
|
LOG.info("region split complete. new regions are: " + regions[0].getRegionName()
|
|
LOG.info("region split complete. new regions are: " + regions[0].getRegionName()
|
|
- + ", " + regions[1].getRegionName());
|
|
|
|
|
|
+ + ", " + regions[1].getRegionName());
|
|
|
|
|
|
return regions;
|
|
return regions;
|
|
}
|
|
}
|
|
@@ -565,10 +566,10 @@ public class HRegion implements HConstants {
|
|
Text key = new Text();
|
|
Text key = new Text();
|
|
long maxSize = 0;
|
|
long maxSize = 0;
|
|
|
|
|
|
- for(Iterator<HStore> i = stores.values().iterator(); i.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStore> i = stores.values().iterator(); i.hasNext(); ) {
|
|
long size = i.next().getLargestFileSize(key);
|
|
long size = i.next().getLargestFileSize(key);
|
|
|
|
|
|
- if (size > maxSize) { // Largest so far
|
|
|
|
|
|
+ if(size > maxSize) { // Largest so far
|
|
maxSize = size;
|
|
maxSize = size;
|
|
midKey.set(key);
|
|
midKey.set(key);
|
|
}
|
|
}
|
|
@@ -593,9 +594,9 @@ public class HRegion implements HConstants {
|
|
public boolean compactStores() throws IOException {
|
|
public boolean compactStores() throws IOException {
|
|
boolean shouldCompact = false;
|
|
boolean shouldCompact = false;
|
|
synchronized(writestate) {
|
|
synchronized(writestate) {
|
|
- if ((!writestate.writesOngoing)
|
|
|
|
|
|
+ if((! writestate.writesOngoing)
|
|
&& writestate.writesEnabled
|
|
&& writestate.writesEnabled
|
|
- && (!writestate.closed)
|
|
|
|
|
|
+ && (! writestate.closed)
|
|
&& recentCommits > MIN_COMMITS_FOR_COMPACTION) {
|
|
&& recentCommits > MIN_COMMITS_FOR_COMPACTION) {
|
|
|
|
|
|
writestate.writesOngoing = true;
|
|
writestate.writesOngoing = true;
|
|
@@ -603,14 +604,14 @@ public class HRegion implements HConstants {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- if (!shouldCompact) {
|
|
|
|
|
|
+ if(! shouldCompact) {
|
|
LOG.info("not compacting region " + this.regionInfo.regionName);
|
|
LOG.info("not compacting region " + this.regionInfo.regionName);
|
|
return false;
|
|
return false;
|
|
|
|
|
|
} else {
|
|
} else {
|
|
try {
|
|
try {
|
|
LOG.info("starting compaction on region " + this.regionInfo.regionName);
|
|
LOG.info("starting compaction on region " + this.regionInfo.regionName);
|
|
- for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
|
HStore store = it.next();
|
|
HStore store = it.next();
|
|
store.compact();
|
|
store.compact();
|
|
}
|
|
}
|
|
@@ -632,7 +633,7 @@ public class HRegion implements HConstants {
|
|
* only take if there have been a lot of uncommitted writes.
|
|
* only take if there have been a lot of uncommitted writes.
|
|
*/
|
|
*/
|
|
public void optionallyFlush() throws IOException {
|
|
public void optionallyFlush() throws IOException {
|
|
- if (commitsSinceFlush > maxUnflushedEntries) {
|
|
|
|
|
|
+ if(commitsSinceFlush > maxUnflushedEntries) {
|
|
flushcache(false);
|
|
flushcache(false);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -657,20 +658,20 @@ public class HRegion implements HConstants {
|
|
public Vector<HStoreFile> flushcache(boolean disableFutureWrites) throws IOException {
|
|
public Vector<HStoreFile> flushcache(boolean disableFutureWrites) throws IOException {
|
|
boolean shouldFlush = false;
|
|
boolean shouldFlush = false;
|
|
synchronized(writestate) {
|
|
synchronized(writestate) {
|
|
- if ((!writestate.writesOngoing)
|
|
|
|
|
|
+ if((! writestate.writesOngoing)
|
|
&& writestate.writesEnabled
|
|
&& writestate.writesEnabled
|
|
- && (!writestate.closed)) {
|
|
|
|
|
|
+ && (! writestate.closed)) {
|
|
|
|
|
|
writestate.writesOngoing = true;
|
|
writestate.writesOngoing = true;
|
|
shouldFlush = true;
|
|
shouldFlush = true;
|
|
|
|
|
|
- if (disableFutureWrites) {
|
|
|
|
|
|
+ if(disableFutureWrites) {
|
|
writestate.writesEnabled = false;
|
|
writestate.writesEnabled = false;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- if (!shouldFlush) {
|
|
|
|
|
|
+ if(! shouldFlush) {
|
|
LOG.debug("not flushing cache for region " + this.regionInfo.regionName);
|
|
LOG.debug("not flushing cache for region " + this.regionInfo.regionName);
|
|
return null;
|
|
return null;
|
|
|
|
|
|
@@ -731,8 +732,8 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
|
|
HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log);
|
|
TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = retval.memcacheSnapshot;
|
|
TreeMap<HStoreKey, BytesWritable> memcacheSnapshot = retval.memcacheSnapshot;
|
|
- if (memcacheSnapshot == null) {
|
|
|
|
- for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ if(memcacheSnapshot == null) {
|
|
|
|
+ for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
|
HStore hstore = it.next();
|
|
HStore hstore = it.next();
|
|
Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
|
|
Vector<HStoreFile> hstoreFiles = hstore.getAllMapFiles();
|
|
allHStoreFiles.addAll(0, hstoreFiles);
|
|
allHStoreFiles.addAll(0, hstoreFiles);
|
|
@@ -746,7 +747,7 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
LOG.debug("flushing memcache to HStores");
|
|
LOG.debug("flushing memcache to HStores");
|
|
|
|
|
|
- for(Iterator<HStore> it = stores.values().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<HStore> it = stores.values().iterator(); it.hasNext(); ) {
|
|
HStore hstore = it.next();
|
|
HStore hstore = it.next();
|
|
Vector<HStoreFile> hstoreFiles
|
|
Vector<HStoreFile> hstoreFiles
|
|
= hstore.flushCache(memcacheSnapshot, logCacheFlushId);
|
|
= hstore.flushCache(memcacheSnapshot, logCacheFlushId);
|
|
@@ -762,7 +763,7 @@ public class HRegion implements HConstants {
|
|
LOG.debug("writing flush cache complete to log");
|
|
LOG.debug("writing flush cache complete to log");
|
|
|
|
|
|
log.completeCacheFlush(this.regionInfo.regionName,
|
|
log.completeCacheFlush(this.regionInfo.regionName,
|
|
- regionInfo.tableDesc.getName(), logCacheFlushId);
|
|
|
|
|
|
+ regionInfo.tableDesc.getName(), logCacheFlushId);
|
|
|
|
|
|
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
|
// C. Delete the now-irrelevant memcache snapshot; its contents have been
|
|
// dumped to disk-based HStores.
|
|
// dumped to disk-based HStores.
|
|
@@ -784,7 +785,7 @@ public class HRegion implements HConstants {
|
|
/** Fetch a single data item. */
|
|
/** Fetch a single data item. */
|
|
public byte[] get(Text row, Text column) throws IOException {
|
|
public byte[] get(Text row, Text column) throws IOException {
|
|
byte results[][] = get(row, column, Long.MAX_VALUE, 1);
|
|
byte results[][] = get(row, column, Long.MAX_VALUE, 1);
|
|
- if (results == null) {
|
|
|
|
|
|
+ if(results == null) {
|
|
return null;
|
|
return null;
|
|
|
|
|
|
} else {
|
|
} else {
|
|
@@ -799,17 +800,16 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
/** Fetch multiple versions of a single data item, with timestamp. */
|
|
/** Fetch multiple versions of a single data item, with timestamp. */
|
|
public byte[][] get(Text row, Text column, long timestamp, int numVersions)
|
|
public byte[][] get(Text row, Text column, long timestamp, int numVersions)
|
|
- throws IOException {
|
|
|
|
|
|
+ throws IOException {
|
|
|
|
|
|
- if (writestate.closed) {
|
|
|
|
|
|
+ if(writestate.closed) {
|
|
throw new IOException("HRegion is closed.");
|
|
throw new IOException("HRegion is closed.");
|
|
}
|
|
}
|
|
|
|
|
|
// Make sure this is a valid row and valid column
|
|
// Make sure this is a valid row and valid column
|
|
|
|
|
|
checkRow(row);
|
|
checkRow(row);
|
|
- Text colFamily = HStoreKey.extractFamily(column);
|
|
|
|
- checkFamily(colFamily);
|
|
|
|
|
|
+ checkColumn(column);
|
|
|
|
|
|
// Obtain the row-lock
|
|
// Obtain the row-lock
|
|
|
|
|
|
@@ -830,7 +830,7 @@ public class HRegion implements HConstants {
|
|
// Check the memcache
|
|
// Check the memcache
|
|
|
|
|
|
byte[][] result = memcache.get(key, numVersions);
|
|
byte[][] result = memcache.get(key, numVersions);
|
|
- if (result != null) {
|
|
|
|
|
|
+ if(result != null) {
|
|
return result;
|
|
return result;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -838,7 +838,7 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
Text colFamily = HStoreKey.extractFamily(key.getColumn());
|
|
Text colFamily = HStoreKey.extractFamily(key.getColumn());
|
|
HStore targetStore = stores.get(colFamily);
|
|
HStore targetStore = stores.get(colFamily);
|
|
- if (targetStore == null) {
|
|
|
|
|
|
+ if(targetStore == null) {
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -859,7 +859,7 @@ public class HRegion implements HConstants {
|
|
HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
|
|
HStoreKey key = new HStoreKey(row, System.currentTimeMillis());
|
|
|
|
|
|
TreeMap<Text, byte[]> memResult = memcache.getFull(key);
|
|
TreeMap<Text, byte[]> memResult = memcache.getFull(key);
|
|
- for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<Text> it = stores.keySet().iterator(); it.hasNext(); ) {
|
|
Text colFamily = it.next();
|
|
Text colFamily = it.next();
|
|
HStore targetStore = stores.get(colFamily);
|
|
HStore targetStore = stores.get(colFamily);
|
|
targetStore.getFull(key, memResult);
|
|
targetStore.getFull(key, memResult);
|
|
@@ -879,7 +879,7 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
HStore storelist[] = new HStore[families.size()];
|
|
HStore storelist[] = new HStore[families.size()];
|
|
int i = 0;
|
|
int i = 0;
|
|
- for(Iterator<Text> it = families.iterator(); it.hasNext();) {
|
|
|
|
|
|
+ for(Iterator<Text> it = families.iterator(); it.hasNext(); ) {
|
|
Text family = it.next();
|
|
Text family = it.next();
|
|
storelist[i++] = stores.get(family);
|
|
storelist[i++] = stores.get(family);
|
|
}
|
|
}
|
|
@@ -911,23 +911,23 @@ public class HRegion implements HConstants {
|
|
/**
|
|
/**
|
|
* Put a cell value into the locked row. The user indicates the row-lock, the
|
|
* Put a cell value into the locked row. The user indicates the row-lock, the
|
|
* target column, and the desired value. This stuff is set into a temporary
|
|
* target column, and the desired value. This stuff is set into a temporary
|
|
- * memory area until the user commits the change, at which pointit's logged
|
|
|
|
|
|
+ * memory area until the user commits the change, at which point it's logged
|
|
* and placed into the memcache.
|
|
* and placed into the memcache.
|
|
*
|
|
*
|
|
* This method really just tests the input, then calls an internal localput()
|
|
* This method really just tests the input, then calls an internal localput()
|
|
* method.
|
|
* method.
|
|
*/
|
|
*/
|
|
public void put(long lockid, Text targetCol, byte[] val) throws IOException {
|
|
public void put(long lockid, Text targetCol, byte[] val) throws IOException {
|
|
- if (val.length == HStoreKey.DELETE_BYTES.length) {
|
|
|
|
|
|
+ if(val.length == HStoreKey.DELETE_BYTES.length) {
|
|
boolean matches = true;
|
|
boolean matches = true;
|
|
for(int i = 0; i < val.length; i++) {
|
|
for(int i = 0; i < val.length; i++) {
|
|
- if (val[i] != HStoreKey.DELETE_BYTES[i]) {
|
|
|
|
|
|
+ if(val[i] != HStoreKey.DELETE_BYTES[i]) {
|
|
matches = false;
|
|
matches = false;
|
|
break;
|
|
break;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- if (matches) {
|
|
|
|
|
|
+ if(matches) {
|
|
throw new IOException("Cannot insert value: " + val);
|
|
throw new IOException("Cannot insert value: " + val);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -950,9 +950,11 @@ public class HRegion implements HConstants {
|
|
* (Or until the user's write-lock expires.)
|
|
* (Or until the user's write-lock expires.)
|
|
*/
|
|
*/
|
|
void localput(long lockid, Text targetCol, byte[] val) throws IOException {
|
|
void localput(long lockid, Text targetCol, byte[] val) throws IOException {
|
|
|
|
+ checkColumn(targetCol);
|
|
|
|
+
|
|
Text row = getRowFromLock(lockid);
|
|
Text row = getRowFromLock(lockid);
|
|
- if (row == null) {
|
|
|
|
- throw new IOException("No write lock for lockid " + lockid);
|
|
|
|
|
|
+ if(row == null) {
|
|
|
|
+ throw new LockException("No write lock for lockid " + lockid);
|
|
}
|
|
}
|
|
|
|
|
|
// This sync block makes localput() thread-safe when multiple
|
|
// This sync block makes localput() thread-safe when multiple
|
|
@@ -964,13 +966,13 @@ public class HRegion implements HConstants {
|
|
// This check makes sure that another thread from the client
|
|
// This check makes sure that another thread from the client
|
|
// hasn't aborted/committed the write-operation.
|
|
// hasn't aborted/committed the write-operation.
|
|
|
|
|
|
- if (row != getRowFromLock(lockid)) {
|
|
|
|
- throw new IOException("Locking error: put operation on lock " + lockid
|
|
|
|
- + " unexpected aborted by another thread");
|
|
|
|
|
|
+ if(row != getRowFromLock(lockid)) {
|
|
|
|
+ throw new LockException("Locking error: put operation on lock " + lockid
|
|
|
|
+ + " unexpected aborted by another thread");
|
|
}
|
|
}
|
|
|
|
|
|
TreeMap<Text, byte[]> targets = targetColumns.get(lockid);
|
|
TreeMap<Text, byte[]> targets = targetColumns.get(lockid);
|
|
- if (targets == null) {
|
|
|
|
|
|
+ if(targets == null) {
|
|
targets = new TreeMap<Text, byte[]>();
|
|
targets = new TreeMap<Text, byte[]>();
|
|
targetColumns.put(lockid, targets);
|
|
targetColumns.put(lockid, targets);
|
|
}
|
|
}
|
|
@@ -985,8 +987,8 @@ public class HRegion implements HConstants {
|
|
*/
|
|
*/
|
|
public void abort(long lockid) throws IOException {
|
|
public void abort(long lockid) throws IOException {
|
|
Text row = getRowFromLock(lockid);
|
|
Text row = getRowFromLock(lockid);
|
|
- if (row == null) {
|
|
|
|
- throw new IOException("No write lock for lockid " + lockid);
|
|
|
|
|
|
+ if(row == null) {
|
|
|
|
+ throw new LockException("No write lock for lockid " + lockid);
|
|
}
|
|
}
|
|
|
|
|
|
// This sync block makes abort() thread-safe when multiple
|
|
// This sync block makes abort() thread-safe when multiple
|
|
@@ -998,9 +1000,9 @@ public class HRegion implements HConstants {
|
|
// This check makes sure another thread from the client
|
|
// This check makes sure another thread from the client
|
|
// hasn't aborted/committed the write-operation.
|
|
// hasn't aborted/committed the write-operation.
|
|
|
|
|
|
- if (row != getRowFromLock(lockid)) {
|
|
|
|
- throw new IOException("Locking error: abort() operation on lock "
|
|
|
|
- + lockid + " unexpected aborted by another thread");
|
|
|
|
|
|
+ if(row != getRowFromLock(lockid)) {
|
|
|
|
+ throw new LockException("Locking error: abort() operation on lock "
|
|
|
|
+ + lockid + " unexpected aborted by another thread");
|
|
}
|
|
}
|
|
|
|
|
|
targetColumns.remove(lockid);
|
|
targetColumns.remove(lockid);
|
|
@@ -1021,8 +1023,8 @@ public class HRegion implements HConstants {
|
|
// that repeated executions won't screw this up.
|
|
// that repeated executions won't screw this up.
|
|
|
|
|
|
Text row = getRowFromLock(lockid);
|
|
Text row = getRowFromLock(lockid);
|
|
- if (row == null) {
|
|
|
|
- throw new IOException("No write lock for lockid " + lockid);
|
|
|
|
|
|
+ if(row == null) {
|
|
|
|
+ throw new LockException("No write lock for lockid " + lockid);
|
|
}
|
|
}
|
|
|
|
|
|
// This check makes sure that another thread from the client
|
|
// This check makes sure that another thread from the client
|
|
@@ -1035,7 +1037,7 @@ public class HRegion implements HConstants {
|
|
|
|
|
|
long commitTimestamp = System.currentTimeMillis();
|
|
long commitTimestamp = System.currentTimeMillis();
|
|
log.append(regionInfo.regionName, regionInfo.tableDesc.getName(), row,
|
|
log.append(regionInfo.regionName, regionInfo.tableDesc.getName(), row,
|
|
- targetColumns.get(lockid), commitTimestamp);
|
|
|
|
|
|
+ targetColumns.get(lockid), commitTimestamp);
|
|
|
|
|
|
memcache.add(row, targetColumns.get(lockid), commitTimestamp);
|
|
memcache.add(row, targetColumns.get(lockid), commitTimestamp);
|
|
|
|
|
|
@@ -1054,25 +1056,26 @@ 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 */
|
|
void checkRow(Text row) throws IOException {
|
|
void checkRow(Text row) throws IOException {
|
|
- if (((regionInfo.startKey.getLength() == 0)
|
|
|
|
- || (regionInfo.startKey.compareTo(row) <= 0))
|
|
|
|
|
|
+ if(((regionInfo.startKey.getLength() == 0)
|
|
|
|
+ || (regionInfo.startKey.compareTo(row) <= 0))
|
|
&& ((regionInfo.endKey.getLength() == 0)
|
|
&& ((regionInfo.endKey.getLength() == 0)
|
|
|| (regionInfo.endKey.compareTo(row) > 0))) {
|
|
|| (regionInfo.endKey.compareTo(row) > 0))) {
|
|
// all's well
|
|
// all's well
|
|
|
|
|
|
} else {
|
|
} else {
|
|
throw new IOException("Requested row out of range for HRegion "
|
|
throw new IOException("Requested row out of range for HRegion "
|
|
- + regionInfo.regionName + ", startKey='" + regionInfo.startKey
|
|
|
|
- + "', endKey='" + regionInfo.endKey + "', row='" + row + "'");
|
|
|
|
|
|
+ + regionInfo.regionName + ", startKey='" + regionInfo.startKey
|
|
|
|
+ + "', endKey='" + regionInfo.endKey + "', row='" + row + "'");
|
|
}
|
|
}
|
|
}
|
|
}
|
|
-
|
|
|
|
|
|
+
|
|
/** Make sure this is a valid column for the current table */
|
|
/** Make sure this is a valid column for the current table */
|
|
- void checkFamily(Text family) throws IOException {
|
|
|
|
- if (!regionInfo.tableDesc.hasFamily(family)) {
|
|
|
|
|
|
+ void checkColumn(Text columnName) throws IOException {
|
|
|
|
+ Text family = new Text(HStoreKey.extractFamily(columnName) + ":");
|
|
|
|
+ if(! regionInfo.tableDesc.hasFamily(family)) {
|
|
throw new IOException("Requested column family " + family
|
|
throw new IOException("Requested column family " + family
|
|
- + " does not exist in HRegion " + regionInfo.regionName
|
|
|
|
- + " for table " + regionInfo.tableDesc.getName());
|
|
|
|
|
|
+ + " does not exist in HRegion " + regionInfo.regionName
|
|
|
|
+ + " for table " + regionInfo.tableDesc.getName());
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
@@ -1092,6 +1095,8 @@ public class HRegion implements HConstants {
|
|
* which maybe we'll do in the future.
|
|
* which maybe we'll do in the future.
|
|
*/
|
|
*/
|
|
long obtainLock(Text row) throws IOException {
|
|
long obtainLock(Text row) throws IOException {
|
|
|
|
+ checkRow(row);
|
|
|
|
+
|
|
synchronized(rowsToLocks) {
|
|
synchronized(rowsToLocks) {
|
|
while(rowsToLocks.get(row) != null) {
|
|
while(rowsToLocks.get(row) != null) {
|
|
try {
|
|
try {
|
|
@@ -1109,6 +1114,8 @@ public class HRegion implements HConstants {
|
|
}
|
|
}
|
|
|
|
|
|
Text getRowFromLock(long lockid) throws IOException {
|
|
Text getRowFromLock(long lockid) throws IOException {
|
|
|
|
+ // Pattern is that all access to rowsToLocks and/or to
|
|
|
|
+ // locksToRows is via a lock on rowsToLocks.
|
|
synchronized(rowsToLocks) {
|
|
synchronized(rowsToLocks) {
|
|
return locksToRows.get(lockid);
|
|
return locksToRows.get(lockid);
|
|
}
|
|
}
|
|
@@ -1150,7 +1157,7 @@ public class HRegion implements HConstants {
|
|
keys[i] = new HStoreKey();
|
|
keys[i] = new HStoreKey();
|
|
resultSets[i] = new TreeMap<Text, byte[]>();
|
|
resultSets[i] = new TreeMap<Text, byte[]>();
|
|
|
|
|
|
- if (!scanners[i].next(keys[i], resultSets[i])) {
|
|
|
|
|
|
+ if(! scanners[i].next(keys[i], resultSets[i])) {
|
|
closeScanner(i);
|
|
closeScanner(i);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1167,7 +1174,7 @@ public class HRegion implements HConstants {
|
|
Text chosenRow = null;
|
|
Text chosenRow = null;
|
|
long chosenTimestamp = -1;
|
|
long chosenTimestamp = -1;
|
|
for(int i = 0; i < keys.length; i++) {
|
|
for(int i = 0; i < keys.length; i++) {
|
|
- if (scanners[i] != null
|
|
|
|
|
|
+ if(scanners[i] != null
|
|
&& (chosenRow == null
|
|
&& (chosenRow == null
|
|
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|
|
|| (keys[i].getRow().compareTo(chosenRow) < 0)
|
|
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
|
|
|| ((keys[i].getRow().compareTo(chosenRow) == 0)
|
|
@@ -1181,21 +1188,21 @@ public class HRegion implements HConstants {
|
|
// Store the key and results for each sub-scanner. Merge them as appropriate.
|
|
// Store the key and results for each sub-scanner. Merge them as appropriate.
|
|
|
|
|
|
boolean insertedItem = false;
|
|
boolean insertedItem = false;
|
|
- if (chosenTimestamp > 0) {
|
|
|
|
|
|
+ if(chosenTimestamp > 0) {
|
|
key.setRow(chosenRow);
|
|
key.setRow(chosenRow);
|
|
key.setVersion(chosenTimestamp);
|
|
key.setVersion(chosenTimestamp);
|
|
key.setColumn(new Text(""));
|
|
key.setColumn(new Text(""));
|
|
|
|
|
|
for(int i = 0; i < scanners.length; i++) {
|
|
for(int i = 0; i < scanners.length; i++) {
|
|
while((scanners[i] != null)
|
|
while((scanners[i] != null)
|
|
- && (keys[i].getRow().compareTo(chosenRow) == 0)
|
|
|
|
- && (keys[i].getTimestamp() == chosenTimestamp)) {
|
|
|
|
|
|
+ && (keys[i].getRow().compareTo(chosenRow) == 0)
|
|
|
|
+ && (keys[i].getTimestamp() == chosenTimestamp)) {
|
|
|
|
|
|
results.putAll(resultSets[i]);
|
|
results.putAll(resultSets[i]);
|
|
insertedItem = true;
|
|
insertedItem = true;
|
|
|
|
|
|
resultSets[i].clear();
|
|
resultSets[i].clear();
|
|
- if (!scanners[i].next(keys[i], resultSets[i])) {
|
|
|
|
|
|
+ if(! scanners[i].next(keys[i], resultSets[i])) {
|
|
closeScanner(i);
|
|
closeScanner(i);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1204,10 +1211,10 @@ public class HRegion implements HConstants {
|
|
// row label, then its timestamp is bad. We need to advance it.
|
|
// row label, then its timestamp is bad. We need to advance it.
|
|
|
|
|
|
while((scanners[i] != null)
|
|
while((scanners[i] != null)
|
|
- && (keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
|
|
|
|
|
+ && (keys[i].getRow().compareTo(chosenRow) <= 0)) {
|
|
|
|
|
|
resultSets[i].clear();
|
|
resultSets[i].clear();
|
|
- if (!scanners[i].next(keys[i], resultSets[i])) {
|
|
|
|
|
|
+ if(! scanners[i].next(keys[i], resultSets[i])) {
|
|
closeScanner(i);
|
|
closeScanner(i);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -1231,7 +1238,7 @@ public class HRegion implements HConstants {
|
|
/** All done with the scanner. */
|
|
/** All done with the scanner. */
|
|
public void close() throws IOException {
|
|
public void close() throws IOException {
|
|
for(int i = 0; i < scanners.length; i++) {
|
|
for(int i = 0; i < scanners.length; i++) {
|
|
- if (scanners[i] != null) {
|
|
|
|
|
|
+ if(scanners[i] != null) {
|
|
closeScanner(i);
|
|
closeScanner(i);
|
|
}
|
|
}
|
|
}
|
|
}
|