diff --git a/CHANGES.txt b/CHANGES.txt index b04a6a22131..4755b7beeba 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -40,6 +40,7 @@ Trunk (unreleased changes) HADOOP-2126 Use Bob Jenkins' hash for bloom filters HADOOP-2157 Make Scanners implement Iterable HADOOP-2176 Htable.deleteAll documentation is ambiguous + HADOOP-2139 (phase 1) Increase parallelism in region servers. Release 0.15.1 Branch 0.15 diff --git a/src/java/org/apache/hadoop/hbase/HAbstractScanner.java b/src/java/org/apache/hadoop/hbase/HAbstractScanner.java index 7fca6d79a37..f63136ff8fc 100644 --- a/src/java/org/apache/hadoop/hbase/HAbstractScanner.java +++ b/src/java/org/apache/hadoop/hbase/HAbstractScanner.java @@ -212,6 +212,9 @@ public abstract class HAbstractScanner implements HInternalScannerInterface { */ public boolean next(HStoreKey key, SortedMap results) throws IOException { + if (scannerClosed) { + return false; + } // Find the next row label (and timestamp) Text chosenRow = null; long chosenTimestamp = -1; @@ -277,6 +280,7 @@ public abstract class HAbstractScanner implements HInternalScannerInterface { return insertedItem; } + /** {@inheritDoc} */ public Iterator>> iterator() { throw new UnsupportedOperationException("Unimplemented serverside. " + "next(HStoreKey, StortedMap(...) is more efficient"); diff --git a/src/java/org/apache/hadoop/hbase/HConstants.java b/src/java/org/apache/hadoop/hbase/HConstants.java index af36df9489b..ce1399ca0d2 100644 --- a/src/java/org/apache/hadoop/hbase/HConstants.java +++ b/src/java/org/apache/hadoop/hbase/HConstants.java @@ -43,7 +43,8 @@ public interface HConstants { /** default host address */ static final String DEFAULT_HOST = "0.0.0.0"; - + + /** default port that the master listens on */ static final int DEFAULT_MASTER_PORT = 60000; /** Default master address */ @@ -164,7 +165,7 @@ public interface HConstants { * commit. */ static final long LATEST_TIMESTAMP = Long.MAX_VALUE; - + /** * Define for 'return-all-versions'. */ diff --git a/src/java/org/apache/hadoop/hbase/HLocking.java b/src/java/org/apache/hadoop/hbase/HLocking.java deleted file mode 100644 index b7fc74b6417..00000000000 --- a/src/java/org/apache/hadoop/hbase/HLocking.java +++ /dev/null @@ -1,107 +0,0 @@ -/** - * Copyright 2007 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.util.concurrent.atomic.AtomicInteger; - -/** - * HLocking is a set of lock primitives that does not rely on a - * particular thread holding the monitor for an object. This is - * especially important when a lock must persist over multiple RPC's - * since there is no guarantee that the same Server thread will handle - * all the RPC's until the lock is released. Not requiring that the locker - * thread is same as unlocking thread is the key distinction between this - * class and {@link java.util.concurrent.locks.ReentrantReadWriteLock}. - * - *

For each independent entity that needs locking, create a new HLocking - * instance. - */ -public class HLocking { - private Integer mutex; - - // If lockers == 0, the lock is unlocked - // If lockers > 0, locked for read - // If lockers == -1 locked for write - - private AtomicInteger lockers; - - /** Constructor */ - public HLocking() { - this.mutex = new Integer(0); - this.lockers = new AtomicInteger(0); - } - - /** - * Caller needs the no-nexclusive read-lock - */ - public void obtainReadLock() { - synchronized(mutex) { - while(lockers.get() < 0) { - try { - mutex.wait(); - } catch(InterruptedException ie) { - // continue - } - } - lockers.incrementAndGet(); - mutex.notifyAll(); - } - } - - /** - * Caller is finished with the non-exclusive read-lock - */ - public void releaseReadLock() { - synchronized(mutex) { - if(lockers.decrementAndGet() < 0) { - throw new IllegalStateException("lockers: " + lockers); - } - mutex.notifyAll(); - } - } - - /** - * Caller needs the exclusive write-lock - */ - public void obtainWriteLock() { - synchronized(mutex) { - while(!lockers.compareAndSet(0, -1)) { - try { - mutex.wait(); - } catch (InterruptedException ie) { - // continue - } - } - mutex.notifyAll(); - } - } - - /** - * Caller is finished with the write lock - */ - public void releaseWriteLock() { - synchronized(mutex) { - if(!lockers.compareAndSet(-1, 0)) { - throw new IllegalStateException("lockers: " + lockers); - } - mutex.notifyAll(); - } - } -} diff --git a/src/java/org/apache/hadoop/hbase/HLog.java b/src/java/org/apache/hadoop/hbase/HLog.java index 6fae19026ce..a313b66efd9 100644 --- a/src/java/org/apache/hadoop/hbase/HLog.java +++ b/src/java/org/apache/hadoop/hbase/HLog.java @@ -409,13 +409,13 @@ public class HLog implements HConstants { * @param timestamp * @throws IOException */ - synchronized void append(Text regionName, Text tableName, Text row, - TreeMap columns, long timestamp) - throws IOException { + synchronized void append(Text regionName, Text tableName, + TreeMap edits) throws IOException { + if (closed) { throw new IOException("Cannot append; log is closed"); } - long seqNum[] = obtainSeqNum(columns.size()); + long seqNum[] = obtainSeqNum(edits.size()); // The 'lastSeqWritten' map holds the sequence number of the oldest // write for each region. When the cache is flushed, the entry for the // region being flushed is removed if the sequence number of the flush @@ -424,10 +424,12 @@ public class HLog implements HConstants { this.lastSeqWritten.put(regionName, Long.valueOf(seqNum[0])); } int counter = 0; - for (Map.Entry es : columns.entrySet()) { + for (Map.Entry es : edits.entrySet()) { + HStoreKey key = es.getKey(); HLogKey logKey = - new HLogKey(regionName, tableName, row, seqNum[counter++]); - HLogEdit logEdit = new HLogEdit(es.getKey(), es.getValue(), timestamp); + new HLogKey(regionName, tableName, key.getRow(), seqNum[counter++]); + HLogEdit logEdit = + new HLogEdit(key.getColumn(), es.getValue(), key.getTimestamp()); this.writer.append(logKey, logEdit); this.numEntries++; } diff --git a/src/java/org/apache/hadoop/hbase/HMaster.java b/src/java/org/apache/hadoop/hbase/HMaster.java index 30c968be7c1..1f8aff7645b 100644 --- a/src/java/org/apache/hadoop/hbase/HMaster.java +++ b/src/java/org/apache/hadoop/hbase/HMaster.java @@ -47,7 +47,6 @@ import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; @@ -2468,10 +2467,12 @@ HMasterRegionInterface { // for the table we want to create already exists, then table already // created. Throw already-exists exception. - MetaRegion m = (onlineMetaRegions.containsKey(newRegion.getRegionName()) ? - onlineMetaRegions.get(newRegion.getRegionName()) : - onlineMetaRegions.get(onlineMetaRegions.headMap( - newRegion.getTableDesc().getName()).lastKey())); + MetaRegion m = (onlineMetaRegions.size() == 1 ? + onlineMetaRegions.get(onlineMetaRegions.firstKey()) : + (onlineMetaRegions.containsKey(newRegion.getRegionName()) ? + onlineMetaRegions.get(newRegion.getRegionName()) : + onlineMetaRegions.get(onlineMetaRegions.headMap( + newRegion.getTableDesc().getName()).lastKey()))); Text metaRegionName = m.getRegionName(); HRegionInterface server = connection.getHRegionConnection(m.getServer()); diff --git a/src/java/org/apache/hadoop/hbase/HMemcache.java b/src/java/org/apache/hadoop/hbase/HMemcache.java deleted file mode 100644 index b989766957c..00000000000 --- a/src/java/org/apache/hadoop/hbase/HMemcache.java +++ /dev/null @@ -1,490 +0,0 @@ -/** - * Copyright 2007 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; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.SortedMap; -import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.io.ImmutableBytesWritable; -import org.apache.hadoop.io.Text; - -/** - * The HMemcache holds in-memory modifications to the HRegion. This is really a - * wrapper around a TreeMap that helps us when staging the Memcache out to disk. - */ -public class HMemcache { - static final Log LOG = LogFactory.getLog(HMemcache.class); - - // Note that since these structures are always accessed with a lock held, - // no additional synchronization is required. - - volatile SortedMap memcache; - List> history = - Collections.synchronizedList(new ArrayList>()); - volatile SortedMap snapshot = null; - - final HLocking lock = new HLocking(); - - /* - * Approximate size in bytes of the payload carried by this memcache. - * Does not consider deletes nor adding again on same key. - */ - private AtomicLong size = new AtomicLong(0); - - - /** - * Constructor - */ - public HMemcache() { - super(); - memcache = - Collections.synchronizedSortedMap(new TreeMap()); - } - - /** represents the state of the memcache at a specified point in time */ - static class Snapshot { - final SortedMap memcacheSnapshot; - final long sequenceId; - - Snapshot(final SortedMap memcache, final Long i) { - super(); - this.memcacheSnapshot = memcache; - this.sequenceId = i.longValue(); - } - } - - /** - * Returns a snapshot of the current HMemcache with a known HLog - * sequence number at the same time. - * - * We need to prevent any writing to the cache during this time, - * so we obtain a write lock for the duration of the operation. - * - *

If this method returns non-null, client must call - * {@link #deleteSnapshot()} to clear 'snapshot-in-progress' - * state when finished with the returned {@link Snapshot}. - * - * @return frozen HMemcache TreeMap and HLog sequence number. - */ - Snapshot snapshotMemcacheForLog(HLog log) throws IOException { - this.lock.obtainWriteLock(); - try { - if(snapshot != null) { - throw new IOException("Snapshot in progress!"); - } - // If no entries in memcache. - if(memcache.size() == 0) { - return null; - } - Snapshot retval = - new Snapshot(memcache, Long.valueOf(log.startCacheFlush())); - // From here on, any failure is catastrophic requiring replay of hlog - this.snapshot = memcache; - synchronized (history) { - history.add(memcache); - } - memcache = - Collections.synchronizedSortedMap(new TreeMap()); - // Reset size of this memcache. - this.size.set(0); - return retval; - } finally { - this.lock.releaseWriteLock(); - } - } - - /** - * Delete the snapshot, remove from history. - * - * Modifying the structure means we need to obtain a writelock. - * @throws IOException - */ - public void deleteSnapshot() throws IOException { - this.lock.obtainWriteLock(); - - try { - if(snapshot == null) { - throw new IOException("Snapshot not present!"); - } - synchronized (history) { - history.remove(snapshot); - } - this.snapshot = null; - } finally { - this.lock.releaseWriteLock(); - } - } - - /** - * Store a value. - * Operation uses a write lock. - * @param row - * @param columns - * @param timestamp - */ - public void add(final Text row, final TreeMap columns, - final long timestamp) { - this.lock.obtainWriteLock(); - try { - for (Map.Entry es: columns.entrySet()) { - HStoreKey key = new HStoreKey(row, es.getKey(), timestamp); - byte [] value = es.getValue(); - this.size.addAndGet(key.getSize()); - this.size.addAndGet(((value == null)? 0: value.length)); - memcache.put(key, value); - } - } finally { - this.lock.releaseWriteLock(); - } - } - - /** - * @return Approximate size in bytes of payload carried by this memcache. - * Does not take into consideration deletes nor adding again on same key. - */ - public long getSize() { - return this.size.get(); - } - - /** - * Look back through all the backlog TreeMaps to find the target. - * @param key - * @param numVersions - * @return An array of byte arrays ordered by timestamp. - */ - public byte [][] get(final HStoreKey key, final int numVersions) { - this.lock.obtainReadLock(); - try { - ArrayList results = get(memcache, key, numVersions); - synchronized (history) { - for (int i = history.size() - 1; i >= 0; i--) { - if (numVersions > 0 && results.size() >= numVersions) { - break; - } - results.addAll(results.size(), - get(history.get(i), key, numVersions - results.size())); - } - } - return (results.size() == 0) ? null : - ImmutableBytesWritable.toArray(results); - - } finally { - this.lock.releaseReadLock(); - } - } - - /** - * Return all the available columns for the given key. The key indicates a - * row and timestamp, but not a column name. - * - * The returned object should map column names to byte arrays (byte[]). - * @param key - * @return All columns for given key. - */ - public TreeMap getFull(HStoreKey key) { - TreeMap results = new TreeMap(); - this.lock.obtainReadLock(); - try { - internalGetFull(memcache, key, results); - synchronized (history) { - for (int i = history.size() - 1; i >= 0; i--) { - SortedMap cur = history.get(i); - internalGetFull(cur, key, results); - } - } - return results; - - } finally { - this.lock.releaseReadLock(); - } - } - - void internalGetFull(SortedMap map, HStoreKey key, - TreeMap results) { - SortedMap tailMap = map.tailMap(key); - for (Map.Entry es: tailMap.entrySet()) { - HStoreKey itKey = es.getKey(); - Text itCol = itKey.getColumn(); - if (results.get(itCol) == null - && key.matchesWithoutColumn(itKey)) { - byte [] val = tailMap.get(itKey); - results.put(itCol, val); - } else if (key.getRow().compareTo(itKey.getRow()) < 0) { - break; - } - } - } - - /** - * Examine a single map for the desired key. - * - * We assume that all locking is done at a higher-level. No locking within - * this method. - * - * TODO - This is kinda slow. We need a data structure that allows for - * proximity-searches, not just precise-matches. - * - * @param map - * @param key - * @param numVersions - * @return Ordered list of items found in passed map. If no - * matching values, returns an empty list (does not return null). - */ - ArrayList get(final SortedMap map, - final HStoreKey key, final int numVersions) { - ArrayList result = new ArrayList(); - // TODO: If get is of a particular version -- numVersions == 1 -- we - // should be able to avoid all of the tailmap creations and iterations - // below. - HStoreKey curKey = new HStoreKey(key); - SortedMap tailMap = map.tailMap(curKey); - for (Map.Entry es: tailMap.entrySet()) { - HStoreKey itKey = es.getKey(); - if (itKey.matchesRowCol(curKey)) { - if (!HLogEdit.isDeleted(es.getValue())) { - result.add(tailMap.get(itKey)); - curKey.setVersion(itKey.getTimestamp() - 1); - } - } - if (numVersions > 0 && result.size() >= numVersions) { - break; - } - } - return result; - } - - /** - * Get versions keys matching the origin key's - * row/column/timestamp and those of an older vintage - * Default access so can be accessed out of {@link HRegionServer}. - * @param origin Where to start searching. - * @param versions How many versions to return. Pass - * {@link HConstants.ALL_VERSIONS} to retrieve all. - * @return Ordered list of versions keys going from newest back. - * @throws IOException - */ - List getKeys(final HStoreKey origin, final int versions) { - this.lock.obtainReadLock(); - try { - List results = getKeys(this.memcache, origin, versions); - synchronized (history) { - for (int i = history.size() - 1; i >= 0; i--) { - results.addAll(results.size(), getKeys(history.get(i), origin, - versions == HConstants.ALL_VERSIONS ? versions : - (versions - results.size()))); - } - } - return results; - } finally { - this.lock.releaseReadLock(); - } - } - - /* - * @param origin Where to start searching. - * @param versions How many versions to return. Pass - * {@link HConstants.ALL_VERSIONS} to retrieve all. - * @return List of all keys that are of the same row and column and of - * equal or older timestamp. If no keys, returns an empty List. Does not - * return null. - */ - private List getKeys(final SortedMap map, - final HStoreKey origin, final int versions) { - List result = new ArrayList(); - SortedMap tailMap = map.tailMap(origin); - for (Map.Entry es: tailMap.entrySet()) { - HStoreKey key = es.getKey(); - if (!key.matchesRowCol(origin)) { - break; - } - if (!HLogEdit.isDeleted(es.getValue())) { - result.add(key); - if (versions != HConstants.ALL_VERSIONS && result.size() >= versions) { - // We have enough results. Return. - break; - } - } - } - return result; - } - - /** - * @param key - * @return True if an entry and its content is {@link HGlobals.deleteBytes}. - * Use checking values in store. On occasion the memcache has the fact that - * the cell has been deleted. - */ - boolean isDeleted(final HStoreKey key) { - return HLogEdit.isDeleted(this.memcache.get(key)); - } - - /** - * Return a scanner over the keys in the HMemcache - */ - HInternalScannerInterface getScanner(long timestamp, - Text targetCols[], Text firstRow) throws IOException { - return new HMemcacheScanner(timestamp, targetCols, firstRow); - } - - ////////////////////////////////////////////////////////////////////////////// - // HMemcacheScanner implements the HScannerInterface. - // It lets the caller scan the contents of the Memcache. - ////////////////////////////////////////////////////////////////////////////// - - class HMemcacheScanner extends HAbstractScanner { - SortedMap backingMaps[]; - final Iterator keyIterators[]; - - @SuppressWarnings("unchecked") - HMemcacheScanner(final long timestamp, final Text targetCols[], - final Text firstRow) throws IOException { - - super(timestamp, targetCols); - lock.obtainReadLock(); - try { - synchronized (history) { - this.backingMaps = new SortedMap[history.size() + 1]; - - // Note that since we iterate through the backing maps from 0 to n, we - // need to put the memcache first, the newest history second, ..., etc. - - backingMaps[0] = memcache; - for (int i = history.size() - 1; i >= 0; i--) { - backingMaps[i + 1] = history.get(i); - } - } - - this.keyIterators = new Iterator[backingMaps.length]; - this.keys = new HStoreKey[backingMaps.length]; - this.vals = new byte[backingMaps.length][]; - - // Generate list of iterators - - HStoreKey firstKey = new HStoreKey(firstRow); - for (int i = 0; i < backingMaps.length; i++) { - if (firstRow != null && firstRow.getLength() != 0) { - keyIterators[i] = - backingMaps[i].tailMap(firstKey).keySet().iterator(); - - } else { - keyIterators[i] = backingMaps[i].keySet().iterator(); - } - - while (getNext(i)) { - if (!findFirstRow(i, firstRow)) { - continue; - } - if (columnMatch(i)) { - break; - } - } - } - } catch (RuntimeException ex) { - LOG.error("error initializing HMemcache scanner: ", ex); - close(); - IOException e = new IOException("error initializing HMemcache scanner"); - e.initCause(ex); - throw e; - - } catch(IOException ex) { - LOG.error("error initializing HMemcache scanner: ", ex); - close(); - throw ex; - } - } - - /** - * The user didn't want to start scanning at the first row. This method - * seeks to the requested row. - * - * @param i which iterator to advance - * @param firstRow seek to this row - * @return true if this is the first row - */ - @Override - boolean findFirstRow(int i, Text firstRow) { - return firstRow.getLength() == 0 || - keys[i].getRow().compareTo(firstRow) >= 0; - } - - /** - * Get the next value from the specified iterator. - * - * @param i Which iterator to fetch next value from - * @return true if there is more data available - */ - @Override - boolean getNext(int i) { - boolean result = false; - while (true) { - if (!keyIterators[i].hasNext()) { - closeSubScanner(i); - break; - } - // Check key is < than passed timestamp for this scanner. - HStoreKey hsk = keyIterators[i].next(); - if (hsk == null) { - throw new NullPointerException("Unexpected null key"); - } - if (hsk.getTimestamp() <= this.timestamp) { - this.keys[i] = hsk; - this.vals[i] = backingMaps[i].get(keys[i]); - result = true; - break; - } - } - return result; - } - - /** Shut down an individual map iterator. */ - @Override - void closeSubScanner(int i) { - keyIterators[i] = null; - keys[i] = null; - vals[i] = null; - backingMaps[i] = null; - } - - /** Shut down map iterators, and release the lock */ - public void close() { - if (!scannerClosed) { - try { - for (int i = 0; i < keys.length; i++) { - if(keyIterators[i] != null) { - closeSubScanner(i); - } - } - } finally { - lock.releaseReadLock(); - scannerClosed = true; - } - } - } - } -} diff --git a/src/java/org/apache/hadoop/hbase/HMerge.java b/src/java/org/apache/hadoop/hbase/HMerge.java index ab83fbb9285..a0cf28b688e 100644 --- a/src/java/org/apache/hadoop/hbase/HMerge.java +++ b/src/java/org/apache/hadoop/hbase/HMerge.java @@ -23,6 +23,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; import java.util.NoSuchElementException; +import java.util.Random; import java.util.TreeMap; import java.util.TreeSet; @@ -34,6 +35,8 @@ import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hbase.io.BatchUpdate; + /** * A non-instantiable class that has a static method capable of compacting * a table by merging adjacent regions that have grown too small. @@ -41,6 +44,7 @@ import org.apache.hadoop.io.Text; class HMerge implements HConstants { static final Log LOG = LogFactory.getLog(HMerge.class); static final Text[] META_COLS = {COL_REGIONINFO}; + static final Random rand = new Random(); private HMerge() { // Not instantiable @@ -366,53 +370,30 @@ class HMerge implements HConstants { oldRegion2 }; for(int r = 0; r < regionsToDelete.length; r++) { - long lockid = -1L; - try { - lockid = root.startUpdate(regionsToDelete[r]); - root.delete(lockid, COL_REGIONINFO); - root.delete(lockid, COL_SERVER); - root.delete(lockid, COL_STARTCODE); - root.commit(lockid, System.currentTimeMillis()); - lockid = -1L; + long lockid = Math.abs(rand.nextLong()); + BatchUpdate b = new BatchUpdate(lockid); + lockid = b.startUpdate(regionsToDelete[r]); + b.delete(lockid, COL_REGIONINFO); + b.delete(lockid, COL_SERVER); + b.delete(lockid, COL_STARTCODE); + root.batchUpdate(System.currentTimeMillis(), b); + lockid = -1L; - if(LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " + regionsToDelete[r]); - } - } finally { - try { - if(lockid != -1L) { - root.abort(lockid); - } - - } catch(IOException iex) { - LOG.error(iex); - } + if(LOG.isDebugEnabled()) { + LOG.debug("updated columns in row: " + regionsToDelete[r]); } } ByteArrayOutputStream byteValue = new ByteArrayOutputStream(); DataOutputStream s = new DataOutputStream(byteValue); newRegion.getRegionInfo().setOffline(true); newRegion.getRegionInfo().write(s); - long lockid = -1L; - try { - lockid = root.startUpdate(newRegion.getRegionName()); - root.put(lockid, COL_REGIONINFO, byteValue.toByteArray()); - root.commit(lockid, System.currentTimeMillis()); - lockid = -1L; - - if(LOG.isDebugEnabled()) { - LOG.debug("updated columns in row: " - + newRegion.getRegionName()); - } - } finally { - try { - if(lockid != -1L) { - root.abort(lockid); - } - - } catch(IOException iex) { - LOG.error(iex); - } + long lockid = Math.abs(rand.nextLong()); + BatchUpdate b = new BatchUpdate(lockid); + lockid = b.startUpdate(newRegion.getRegionName()); + b.put(lockid, COL_REGIONINFO, byteValue.toByteArray()); + root.batchUpdate(System.currentTimeMillis(), b); + if(LOG.isDebugEnabled()) { + LOG.debug("updated columns in row: " + newRegion.getRegionName()); } } } diff --git a/src/java/org/apache/hadoop/hbase/HRegion.java b/src/java/org/apache/hadoop/hbase/HRegion.java index 91ebe727035..f4ff1a482f7 100644 --- a/src/java/org/apache/hadoop/hbase/HRegion.java +++ b/src/java/org/apache/hadoop/hbase/HRegion.java @@ -32,12 +32,17 @@ import java.util.TreeSet; import java.util.Vector; import java.util.Map.Entry; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantReadWriteLock; 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.hbase.filter.RowFilterInterface; +import org.apache.hadoop.hbase.io.BatchOperation; +import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.util.Writables; import org.apache.hadoop.io.Text; import org.apache.hadoop.util.StringUtils; @@ -58,8 +63,14 @@ import org.apache.hadoop.util.StringUtils; * startKey for region 1 (if it exists). The startKey for the * first region is null. The endKey for the final region is null. * - *

The HStores have no locking built-in. All row-level locking - * and row-level atomicity is provided by the HRegion. + *

Locking at the HRegion level serves only one purpose: preventing the + * region from being closed (and consequently split) while other operations + * are ongoing. Each row level operation obtains both a row lock and a region + * read lock for the duration of the operation. While a scanner is being + * constructed, getScanner holds a read lock. If the scanner is successfully + * constructed, it holds a read lock until it is closed. A close takes out a + * write lock and consequently will block for ongoing operations and will block + * new operations from starting while the close is in progress. * *

An HRegion is defined by its table and its key extent. * @@ -74,13 +85,13 @@ import org.apache.hadoop.util.StringUtils; * defines the keyspace for this HRegion. */ public class HRegion implements HConstants { - static String SPLITDIR = "splits"; - static String MERGEDIR = "merges"; + static final String SPLITDIR = "splits"; + static final String MERGEDIR = "merges"; static final Random rand = new Random(); static final Log LOG = LogFactory.getLog(HRegion.class); final AtomicBoolean closed = new AtomicBoolean(false); - private long noFlushCount = 0; - + private volatile long noFlushCount = 0; + /** * Merge two HRegions. They must be available on the current * HRegionServer. Returns a brand-new active HRegion, also @@ -88,7 +99,7 @@ public class HRegion implements HConstants { */ static HRegion closeAndMerge(final HRegion srcA, final HRegion srcB) throws IOException { - + HRegion a = srcA; HRegion b = srcB; @@ -105,7 +116,7 @@ public class HRegion implements HConstants { a = srcB; b = srcA; } - + if (! a.getEndKey().equals(b.getStartKey())) { throw new IOException("Cannot merge non-adjacent regions"); } @@ -120,17 +131,17 @@ public class HRegion implements HConstants { if(! fs.exists(merges)) { fs.mkdirs(merges); } - + HRegionInfo newRegionInfo = new HRegionInfo(tabledesc, startKey, endKey); Path newRegionDir = HRegion.getRegionDir(merges, HRegionInfo.encodeRegionName(newRegionInfo.getRegionName())); if(fs.exists(newRegionDir)) { throw new IOException("Cannot merge; target file collision at " + - newRegionDir); + newRegionDir); } LOG.info("starting merge of regions: " + a.getRegionName() + " and " + - b.getRegionName() + " into new region " + newRegionInfo.toString()); + b.getRegionName() + " into new region " + newRegionInfo.toString()); Map> byFamily = new TreeMap>(); @@ -151,14 +162,14 @@ public class HRegion implements HConstants { newRegionDir); // Get rid of merges directory - + fs.delete(merges); LOG.info("merge completed. New region is " + dstRegion.getRegionName()); - + return dstRegion; } - + /* * Fills a map with a vector of store files keyed by column family. * @param byFamily Map to fill. @@ -182,20 +193,20 @@ public class HRegion implements HConstants { // Members ////////////////////////////////////////////////////////////////////////////// - Map rowsToLocks = new HashMap(); - Map locksToRows = new HashMap(); - Map stores = new HashMap(); - Map> targetColumns - = new HashMap>(); - - final HMemcache memcache; + volatile Map rowsToLocks = new HashMap(); + volatile Map locksToRows = new HashMap(); + volatile Map stores = new HashMap(); + volatile Map> targetColumns = + new HashMap>(); - Path rootDir; - HLog log; - FileSystem fs; - HBaseConfiguration conf; - HRegionInfo regionInfo; - Path regiondir; + final AtomicLong memcacheSize = new AtomicLong(0); + + final Path rootDir; + final HLog log; + final FileSystem fs; + final HBaseConfiguration conf; + final HRegionInfo regionInfo; + final Path regiondir; static class WriteState { // Set while a memcache flush is happening. @@ -206,17 +217,19 @@ public class HRegion implements HConstants { // again. volatile boolean writesEnabled = true; } - + volatile WriteState writestate = new WriteState(); final int memcacheFlushSize; final int blockingMemcacheSize; protected final long threadWakeFrequency; protected final int optionalFlushCount; - private final HLocking lock = new HLocking(); - private long desiredMaxFileSize; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final Integer updateLock = new Integer(0); + private final long desiredMaxFileSize; private final long minSequenceId; private final String encodedRegionName; + final AtomicInteger activeScannerCount = new AtomicInteger(0); ////////////////////////////////////////////////////////////////////////////// // Constructor @@ -242,8 +255,8 @@ public class HRegion implements HConstants { * @throws IOException */ public HRegion(Path rootDir, HLog log, FileSystem fs, HBaseConfiguration conf, - HRegionInfo regionInfo, Path initialFiles) - throws IOException { + HRegionInfo regionInfo, Path initialFiles) throws IOException { + this.rootDir = rootDir; this.log = log; this.fs = fs; @@ -251,7 +264,6 @@ public class HRegion implements HConstants { this.regionInfo = regionInfo; this.encodedRegionName = HRegionInfo.encodeRegionName(this.regionInfo.getRegionName()); - this.memcache = new HMemcache(); this.threadWakeFrequency = conf.getLong(THREAD_WAKE_FREQUENCY, 10 * 1000); this.optionalFlushCount = conf.getInt("hbase.hregion.memcache.optionalflushcount", 10); @@ -270,14 +282,14 @@ public class HRegion implements HConstants { // Load in all the HStores. long maxSeqId = -1; for(Map.Entry e : - this.regionInfo.getTableDesc().families().entrySet()) { + this.regionInfo.getTableDesc().families().entrySet()) { Text colFamily = HStoreKey.extractFamily(e.getKey()); - + HStore store = new HStore(rootDir, this.regionInfo.getRegionName(), this.encodedRegionName, e.getValue(), fs, oldLogFile, conf); - + stores.put(colFamily, store); - + long storeSeqId = store.getMaxSequenceId(); if (storeSeqId > maxSeqId) { maxSeqId = storeSeqId; @@ -286,7 +298,7 @@ public class HRegion implements HConstants { this.minSequenceId = maxSeqId; if (LOG.isDebugEnabled()) { LOG.debug("Next sequence id for region " + regionInfo.getRegionName() + - " is " + this.minSequenceId); + " is " + this.minSequenceId); } // Get rid of any splits or merges that were lost in-progress @@ -301,10 +313,10 @@ public class HRegion implements HConstants { // By default, we flush the cache when 16M. this.memcacheFlushSize = conf.getInt("hbase.hregion.memcache.flush.size", - 1024*1024*16); + 1024*1024*16); this.blockingMemcacheSize = this.memcacheFlushSize * conf.getInt("hbase.hregion.memcache.block.multiplier", 2); - + // By default we split region if a file > DEFAULT_MAX_FILE_SIZE. this.desiredMaxFileSize = conf.getLong("hbase.hregion.max.filesize", DEFAULT_MAX_FILE_SIZE); @@ -368,10 +380,10 @@ public class HRegion implements HConstants { LOG.info("region " + this.regionInfo.getRegionName() + " already closed"); return null; } - lock.obtainWriteLock(); + lock.writeLock().lock(); try { - synchronized(writestate) { - while(writestate.compacting || writestate.flushing) { + synchronized (writestate) { + while (writestate.compacting || writestate.flushing) { try { writestate.wait(); } catch (InterruptedException iex) { @@ -383,14 +395,28 @@ public class HRegion implements HConstants { writestate.writesEnabled = false; } + // Wait for active scanners to finish. The write lock we hold will prevent + // new scanners from being created. + + synchronized (activeScannerCount) { + while (activeScannerCount.get() != 0) { + try { + activeScannerCount.wait(); + + } catch (InterruptedException e) { + // continue + } + } + } + // Write lock means no more row locks can be given out. Wait on // outstanding row locks to come in before we close so we do not drop // outstanding updates. waitOnRowLocks(); + // Don't flush the cache if we are aborting if (!abort) { - // Don't flush the cache if we are aborting during a test. - internalFlushcache(); + internalFlushcache(snapshotMemcaches()); } Vector result = new Vector(); @@ -401,24 +427,116 @@ public class HRegion implements HConstants { LOG.info("closed " + this.regionInfo.getRegionName()); return result; } finally { - lock.releaseWriteLock(); + lock.writeLock().unlock(); } } + ////////////////////////////////////////////////////////////////////////////// + // HRegion accessors + ////////////////////////////////////////////////////////////////////////////// + + /** @return start key for region */ + public Text getStartKey() { + return this.regionInfo.getStartKey(); + } + + /** @return end key for region */ + public Text getEndKey() { + return this.regionInfo.getEndKey(); + } + + /** @return region id */ + public long getRegionId() { + return this.regionInfo.getRegionId(); + } + + /** @return region name */ + public Text getRegionName() { + return this.regionInfo.getRegionName(); + } + + /** @return root directory path */ + public Path getRootDir() { + return rootDir; + } + + /** @return HTableDescriptor for this region */ + public HTableDescriptor getTableDesc() { + return this.regionInfo.getTableDesc(); + } + + /** @return HLog in use for this region */ + public HLog getLog() { + return this.log; + } + + /** @return Configuration object */ + public HBaseConfiguration getConf() { + return this.conf; + } + + /** @return region directory Path */ + public Path getRegionDir() { + return this.regiondir; + } + + /** @return FileSystem being used by this region */ + public FileSystem getFilesystem() { + return this.fs; + } + + ////////////////////////////////////////////////////////////////////////////// + // HRegion maintenance. + // + // These methods are meant to be called periodically by the HRegionServer for + // upkeep. + ////////////////////////////////////////////////////////////////////////////// + + /** + * @return returns size of largest HStore. Also returns whether store is + * splitable or not (Its not splitable if region has a store that has a + * reference store file). + */ + HStore.HStoreSize largestHStore(Text midkey) { + HStore.HStoreSize biggest = null; + boolean splitable = true; + for(HStore h: stores.values()) { + HStore.HStoreSize size = h.size(midkey); + // If we came across a reference down in the store, then propagate + // fact that region is not splitable. + if (splitable) { + splitable = size.splitable; + } + if (biggest == null) { + biggest = size; + continue; + } + if(size.getAggregate() > biggest.getAggregate()) { // Largest so far + biggest = size; + } + } + if (biggest != null) { + biggest.setSplitable(splitable); + } + return biggest; + } + /* * Split the HRegion to create two brand-new ones. This also closes * current HRegion. Split should be fast since we don't rewrite store files * but instead create new 'reference' store files that read off the top and * bottom ranges of parent store files. - * @param midKey Row to split on. * @param listener May be null. - * @return two brand-new (and open) HRegions + * @return two brand-new (and open) HRegions or null if a split is not needed * @throws IOException */ - HRegion[] closeAndSplit(final Text midKey, - final RegionUnavailableListener listener) - throws IOException { - checkMidKey(midKey); + HRegion[] splitRegion(final RegionUnavailableListener listener) + throws IOException { + + Text midKey = new Text(); + if (!needsSplit(midKey)) { + return null; + } long startTime = System.currentTimeMillis(); Path splits = getSplitsDir(); HRegionInfo regionAInfo = new HRegionInfo(this.regionInfo.getTableDesc(), @@ -496,14 +614,46 @@ public class HRegion implements HConstants { return regions; } - private void checkMidKey(final Text midKey) throws IOException { - if(((this.regionInfo.getStartKey().getLength() != 0) - && (this.regionInfo.getStartKey().compareTo(midKey) > 0)) - || ((this.regionInfo.getEndKey().getLength() != 0) - && (this.regionInfo.getEndKey().compareTo(midKey) < 0))) { - throw new IOException("Region splitkey must lie within region " + - "boundaries."); + /* + * Iterates through all the HStores and finds the one with the largest + * MapFile size. If the size is greater than the (currently hard-coded) + * threshold, returns true indicating that the region should be split. The + * midKey for the largest MapFile is returned through the midKey parameter. + * It is possible for us to rule the region non-splitable even in excess of + * configured size. This happens if region contains a reference file. If + * a reference file, the region can not be split. + * + * Note that there is no need to do locking in this method because it calls + * largestHStore which does the necessary locking. + * + * @param midKey midKey of the largest MapFile + * @return true if the region should be split. midKey is set by this method. + * Check it for a midKey value on return. + */ + boolean needsSplit(Text midKey) { + HStore.HStoreSize biggest = largestHStore(midKey); + if (biggest == null) { + return false; } + long triggerSize = this.desiredMaxFileSize + (this.desiredMaxFileSize / 2); + boolean split = (biggest.getAggregate() >= triggerSize); + if (split) { + if (!biggest.isSplitable()) { + LOG.warn("Region " + getRegionName().toString() + + " is NOT splitable though its aggregate size is " + + StringUtils.humanReadableInt(biggest.getAggregate()) + + " and desired size is " + + StringUtils.humanReadableInt(this.desiredMaxFileSize)); + split = false; + } else { + LOG.info("Splitting " + getRegionName().toString() + + " because largest aggregate size is " + + StringUtils.humanReadableInt(biggest.getAggregate()) + + " and desired size is " + + StringUtils.humanReadableInt(this.desiredMaxFileSize)); + } + } + return split; } private Path getSplitRegionDir(final Path splits, final String region) { @@ -518,164 +668,31 @@ public class HRegion implements HConstants { return splits; } - ////////////////////////////////////////////////////////////////////////////// - // HRegion accessors - ////////////////////////////////////////////////////////////////////////////// - - /** @return start key for region */ - public Text getStartKey() { - return this.regionInfo.getStartKey(); - } - - /** @return end key for region */ - public Text getEndKey() { - return this.regionInfo.getEndKey(); - } - - /** @return region id */ - public long getRegionId() { - return this.regionInfo.getRegionId(); - } - - /** @return region name */ - public Text getRegionName() { - return this.regionInfo.getRegionName(); - } - - /** @return root directory path */ - public Path getRootDir() { - return rootDir; - } - - /** @return HTableDescriptor for this region */ - public HTableDescriptor getTableDesc() { - return this.regionInfo.getTableDesc(); - } - - /** @return HLog in use for this region */ - public HLog getLog() { - return this.log; - } - - /** @return Configuration object */ - public HBaseConfiguration getConf() { - return this.conf; - } - - /** @return region directory Path */ - public Path getRegionDir() { - return this.regiondir; - } - - /** @return FileSystem being used by this region */ - public FileSystem getFilesystem() { - return this.fs; - } - - ////////////////////////////////////////////////////////////////////////////// - // HRegion maintenance. - // - // These methods are meant to be called periodically by the HRegionServer for - // upkeep. - ////////////////////////////////////////////////////////////////////////////// - - /* - * Iterates through all the HStores and finds the one with the largest - * MapFile size. If the size is greater than the (currently hard-coded) - * threshold, returns true indicating that the region should be split. The - * midKey for the largest MapFile is returned through the midKey parameter. - * It is possible for us to rule the region non-splitable even in excess of - * configured size. This happens if region contains a reference file. If - * a reference file, the region can not be split. - * @param midKey midKey of the largest MapFile - * @return true if the region should be split. midKey is set by this method. - * Check it for a midKey value on return. - */ - boolean needsSplit(Text midKey) { - lock.obtainReadLock(); - try { - HStore.HStoreSize biggest = largestHStore(midKey); - if (biggest == null) { - return false; - } - long triggerSize = - this.desiredMaxFileSize + (this.desiredMaxFileSize / 2); - boolean split = (biggest.getAggregate() >= triggerSize); - if (split) { - if (!biggest.isSplitable()) { - LOG.warn("Region " + getRegionName().toString() + - " is NOT splitable though its aggregate size is " + - StringUtils.humanReadableInt(biggest.getAggregate()) + - " and desired size is " + - StringUtils.humanReadableInt(this.desiredMaxFileSize)); - split = false; - } else { - LOG.info("Splitting " + getRegionName().toString() + - " because largest aggregate size is " + - StringUtils.humanReadableInt(biggest.getAggregate()) + - " and desired size is " + - StringUtils.humanReadableInt(this.desiredMaxFileSize)); - } - } - return split; - } finally { - lock.releaseReadLock(); - } - } - /** - * @return returns size of largest HStore. Also returns whether store is - * splitable or not (Its not splitable if region has a store that has a - * reference store file). + * Only do a compaction if it is necessary + * + * @return + * @throws IOException */ - HStore.HStoreSize largestHStore(final Text midkey) { - HStore.HStoreSize biggest = null; - boolean splitable = true; - lock.obtainReadLock(); - try { - for(HStore h: stores.values()) { - HStore.HStoreSize size = h.size(midkey); - // If we came across a reference down in the store, then propagate - // fact that region is not splitable. - if (splitable) { - splitable = size.splitable; - } - if (biggest == null) { - biggest = size; - continue; - } - if(size.getAggregate() > biggest.getAggregate()) { // Largest so far - biggest = size; - } - } - if (biggest != null) { - biggest.setSplitable(splitable); - } - return biggest; - - } finally { - lock.releaseReadLock(); - } - } - - /** - * @return true if the region should be compacted. - */ - boolean needsCompaction() { + boolean compactIfNeeded() throws IOException { boolean needsCompaction = false; - this.lock.obtainReadLock(); - try { - for (HStore store: stores.values()) { - if (store.needsCompaction()) { - needsCompaction = true; + for (HStore store: stores.values()) { + if (store.needsCompaction()) { + needsCompaction = true; + if (LOG.isDebugEnabled()) { LOG.debug(store.toString() + " needs compaction"); - break; } + break; } - } finally { - this.lock.releaseReadLock(); } - return needsCompaction; + if (!needsCompaction) { + if (LOG.isDebugEnabled()) { + LOG.debug("region " + regionInfo.getRegionName() + + " does not need compaction"); + } + return false; + } + return compactStores(); } /** @@ -689,41 +706,43 @@ public class HRegion implements HConstants { * compaction was not carried out, because the HRegion is busy doing * something else storage-intensive (like flushing the cache). The caller * should check back later. + * + * Note that no locking is necessary at this level because compaction only + * conflicts with a region split, and that cannot happen because the region + * server does them sequentially and not in parallel. */ boolean compactStores() throws IOException { - boolean shouldCompact = false; if (this.closed.get()) { - return shouldCompact; + return false; } - lock.obtainReadLock(); try { synchronized (writestate) { - if ((!writestate.compacting) && - writestate.writesEnabled) { + if (!writestate.compacting && writestate.writesEnabled) { writestate.compacting = true; - shouldCompact = true; + + } else { + LOG.info("NOT compacting region " + + this.regionInfo.getRegionName().toString() + ": compacting=" + + writestate.compacting + ", writesEnabled=" + + writestate.writesEnabled); + return false; } } - - if (!shouldCompact) { - LOG.info("NOT compacting region " + - this.regionInfo.getRegionName().toString()); - return false; - } - long startTime = System.currentTimeMillis(); LOG.info("starting compaction on region " + this.regionInfo.getRegionName().toString()); + boolean status = true; for (HStore store : stores.values()) { - store.compact(); + if(!store.compact()) { + status = false; + } } LOG.info("compaction completed on region " + this.regionInfo.getRegionName().toString() + ". Took " + StringUtils.formatTimeDiff(System.currentTimeMillis(), startTime)); - return true; + return status; } finally { - lock.releaseReadLock(); synchronized (writestate) { writestate.compacting = false; writestate.notifyAll(); @@ -732,31 +751,8 @@ public class HRegion implements HConstants { } /** - * Each HRegion is given a periodic chance to flush the cache, which it should - * only take if there have been a lot of uncommitted writes. - * @throws IOException - * @throws DroppedSnapshotException Thrown when replay of hlog is required - * because a Snapshot was not properly persisted. - */ - void optionallyFlush() throws IOException { - if(this.memcache.getSize() > this.memcacheFlushSize) { - flushcache(false); - } else if (this.memcache.getSize() > 0) { - if (this.noFlushCount >= this.optionalFlushCount) { - LOG.info("Optional flush called " + this.noFlushCount + - " times when data present without flushing. Forcing one."); - flushcache(false); - } else { - // Only increment if something in the cache. - // Gets zero'd when a flushcache is called. - this.noFlushCount++; - } - } - } - - /** - * Flush the cache. This is called periodically to minimize the amount of - * log processing needed upon startup. + * Flush the cache if necessary. This is called periodically to minimize the + * amount of log processing needed upon startup. * *

The returned Vector is a list of all the files used by the component * HStores. It is a list of HStoreFile objects. If the returned value is @@ -775,39 +771,91 @@ public class HRegion implements HConstants { * @throws DroppedSnapshotException Thrown when replay of hlog is required * because a Snapshot was not properly persisted. */ - void flushcache(boolean disableFutureWrites) - throws IOException { - if (this.closed.get()) { - return; - } - this.noFlushCount = 0; - boolean shouldFlush = false; - synchronized(writestate) { - if((!writestate.flushing) && writestate.writesEnabled) { - writestate.flushing = true; - shouldFlush = true; - if(disableFutureWrites) { - writestate.writesEnabled = false; + void flushcache() throws IOException { + lock.readLock().lock(); // Prevent splits and closes + try { + if (this.closed.get()) { + return; + } + boolean needFlush = false; + long memcacheSize = this.memcacheSize.get(); + if(memcacheSize > this.memcacheFlushSize) { + needFlush = true; + } else if (memcacheSize > 0) { + if (this.noFlushCount >= this.optionalFlushCount) { + LOG.info("Optional flush called " + this.noFlushCount + + " times when data present without flushing. Forcing one."); + needFlush = true; + } else { + // Only increment if something in the cache. + // Gets zero'd when a flushcache is called. + this.noFlushCount++; } } + if (!needFlush) { + if (LOG.isDebugEnabled()) { + LOG.debug("Cache flush not needed for region " + + regionInfo.getRegionName() + ". Cache size=" + memcacheSize + + ", cache flush threshold=" + this.memcacheFlushSize); + } + return; + } + synchronized (writestate) { + if ((!writestate.flushing) && writestate.writesEnabled) { + writestate.flushing = true; + + } else { + if(LOG.isDebugEnabled()) { + LOG.debug("NOT flushing memcache for region " + + this.regionInfo.getRegionName() + ", flushing=" + + writestate.flushing + ", writesEnabled=" + + writestate.writesEnabled); + } + return; + } + } + this.noFlushCount = 0; + long startTime = -1; + synchronized (updateLock) {// Stop updates while we snapshot the memcaches + startTime = snapshotMemcaches(); + } + try { + internalFlushcache(startTime); + } finally { + synchronized (writestate) { + writestate.flushing = false; + writestate.notifyAll(); + } + } + } finally { + lock.readLock().unlock(); + } + } + + /* + * It is assumed that updates are blocked for the duration of this method + */ + long snapshotMemcaches() { + if (this.memcacheSize.get() == 0) { + return -1; + } + long startTime = System.currentTimeMillis(); + + if(LOG.isDebugEnabled()) { + LOG.debug("Started memcache flush for region " + + this.regionInfo.getRegionName() + ". Size " + + StringUtils.humanReadableInt(this.memcacheSize.get())); } - if(!shouldFlush) { - if(LOG.isDebugEnabled()) { - LOG.debug("NOT flushing memcache for region " + - this.regionInfo.getRegionName()); - } - return; - } + // We reset the aggregate memcache size here so that subsequent updates + // will add to the unflushed size - try { - internalFlushcache(); - } finally { - synchronized (writestate) { - writestate.flushing = false; - writestate.notifyAll(); - } + this.memcacheSize.set(0L); + + for (HStore hstore: stores.values()) { + hstore.snapshotMemcache(); } + return startTime; } /** @@ -839,13 +887,13 @@ public class HRegion implements HConstants { * @throws DroppedSnapshotException Thrown when replay of hlog is required * because a Snapshot was not properly persisted. */ - void internalFlushcache() throws IOException { - long startTime = -1; - if(LOG.isDebugEnabled()) { - startTime = System.currentTimeMillis(); - LOG.debug("Started memcache flush for region " + - this.regionInfo.getRegionName() + ". Size " + - StringUtils.humanReadableInt(this.memcache.getSize())); + void internalFlushcache(long startTime) throws IOException { + if (startTime == -1) { + if (LOG.isDebugEnabled()) { + LOG.debug("Not flushing cache: snapshotMemcaches() determined that " + + "there was nothing to do"); + } + return; } // We pass the log to the HMemcache, so we can lock down both @@ -859,64 +907,47 @@ public class HRegion implements HConstants { // explicitly cleaned up using a call to deleteSnapshot() or by calling // abort. // - HMemcache.Snapshot retval = memcache.snapshotMemcacheForLog(log); - if(retval == null || retval.memcacheSnapshot == null) { - LOG.debug("Finished memcache flush; empty snapshot"); - return; - } + long sequenceId = log.startCacheFlush(); // Any failure from here on out will be catastrophic requiring server // restart so hlog content can be replayed and put back into the memcache. // Otherwise, the snapshot content while backed up in the hlog, it will not // be part of the current running servers state. + + long logCacheFlushId = sequenceId; try { - long logCacheFlushId = retval.sequenceId; - if(LOG.isDebugEnabled()) { - LOG.debug("Snapshotted memcache for region " + - this.regionInfo.getRegionName() + " with sequence id " + - retval.sequenceId + " and entries " + - retval.memcacheSnapshot.size()); + // A. Flush memcache to all the HStores. + // Keep running vector of all store files that includes both old and the + // just-made new flush store file. + for (HStore hstore: stores.values()) { + hstore.flushCache(sequenceId); } - - try { - // A. Flush memcache to all the HStores. - // Keep running vector of all store files that includes both old and the - // just-made new flush store file. - for (HStore hstore: stores.values()) { - hstore.flushCache(retval.memcacheSnapshot, retval.sequenceId); - } - } catch (IOException e) { - // An exception here means that the snapshot was not persisted. - // The hlog needs to be replayed so its content is restored to memcache. - // Currently, only a server restart will do this. - this.log.abortCacheFlush(); - throw new DroppedSnapshotException(e.getMessage()); - } - - // If we get to here, the HStores have been written. If we get an - // error in completeCacheFlush it will release the lock it is holding - - // B. Write a FLUSHCACHE-COMPLETE message to the log. - // This tells future readers that the HStores were emitted correctly, - // and that all updates to the log for this regionName that have lower - // log-sequence-ids can be safely ignored. - this.log.completeCacheFlush(this.regionInfo.getRegionName(), - regionInfo.getTableDesc().getName(), logCacheFlushId); - - } finally { - // C. Delete the now-irrelevant memcache snapshot; its contents have been - // dumped to disk-based HStores or, if error, clear aborted snapshot. - this.memcache.deleteSnapshot(); + } catch (IOException e) { + // An exception here means that the snapshot was not persisted. + // The hlog needs to be replayed so its content is restored to memcache. + // Currently, only a server restart will do this. + this.log.abortCacheFlush(); + throw new DroppedSnapshotException(e.getMessage()); } - + + // If we get to here, the HStores have been written. If we get an + // error in completeCacheFlush it will release the lock it is holding + + // B. Write a FLUSHCACHE-COMPLETE message to the log. + // This tells future readers that the HStores were emitted correctly, + // and that all updates to the log for this regionName that have lower + // log-sequence-ids can be safely ignored. + this.log.completeCacheFlush(this.regionInfo.getRegionName(), + regionInfo.getTableDesc().getName(), logCacheFlushId); + // D. Finally notify anyone waiting on memcache to clear: // e.g. checkResources(). - synchronized(this) { + synchronized (this) { notifyAll(); } if (LOG.isDebugEnabled()) { LOG.debug("Finished memcache flush for region " + - this.regionInfo.getRegionName() + " in " + + this.regionInfo.getRegionName() + " in " + (System.currentTimeMillis() - startTime) + "ms"); } } @@ -925,20 +956,44 @@ public class HRegion implements HConstants { // get() methods for client use. ////////////////////////////////////////////////////////////////////////////// - /** Fetch a single data item. */ - byte [] get(Text row, Text column) throws IOException { + /** + * Fetch a single data item. + * @param row + * @param column + * @return column value + * @throws IOException + */ + public byte [] get(Text row, Text column) throws IOException { byte [][] results = get(row, column, Long.MAX_VALUE, 1); return (results == null || results.length == 0)? null: results[0]; } - /** Fetch multiple versions of a single data item */ - byte [][] get(Text row, Text column, int numVersions) throws IOException { + /** + * Fetch multiple versions of a single data item + * + * @param row + * @param column + * @param numVersions + * @return array of values one element per version + * @throws IOException + */ + public byte [][] get(Text row, Text column, int numVersions) throws IOException { return get(row, column, Long.MAX_VALUE, numVersions); } - /** Fetch multiple versions of a single data item, with timestamp. */ - byte [][] get(Text row, Text column, long timestamp, int numVersions) - throws IOException { + /** + * Fetch multiple versions of a single data item, with timestamp. + * + * @param row + * @param column + * @param timestamp + * @param numVersions + * @return array of values one element per version that matches the timestamp + * @throws IOException + */ + public byte [][] get(Text row, Text column, long timestamp, int numVersions) + throws IOException { + if (this.closed.get()) { throw new IOException("Region " + this.getRegionName().toString() + " closed"); @@ -948,61 +1003,11 @@ public class HRegion implements HConstants { checkRow(row); checkColumn(column); - // Obtain the row-lock - obtainRowLock(row); - try { - // Obtain the -col results - return get(new HStoreKey(row, column, timestamp), numVersions); + // Don't need a row lock for a simple get - } finally { - releaseRowLock(row); - } - } - - private byte [][] get(final HStoreKey key, final int numVersions) - throws IOException { - lock.obtainReadLock(); - try { - // Check the memcache - byte [][] memcacheResult = this.memcache.get(key, numVersions); - // If we got sufficient versions from memcache, return. - if (memcacheResult != null && memcacheResult.length == numVersions) { - return memcacheResult; - } - - // Check hstore for more versions. - Text colFamily = HStoreKey.extractFamily(key.getColumn()); - HStore targetStore = stores.get(colFamily); - if(targetStore == null) { - // There are no stores. Return what we got from memcache. - return memcacheResult; - } - - // Update the number of versions we need to fetch from the store. - int amendedNumVersions = numVersions; - if (memcacheResult != null) { - amendedNumVersions -= memcacheResult.length; - } - byte [][] result = - targetStore.get(key, amendedNumVersions, this.memcache); - if (result == null) { - result = memcacheResult; - } else if (memcacheResult != null) { - // We have results from both memcache and from stores. Put them - // together in an array in the proper order. - byte [][] storeResult = result; - result = new byte [memcacheResult.length + result.length][]; - for (int i = 0; i < memcacheResult.length; i++) { - result[i] = memcacheResult[i]; - } - for (int i = 0; i < storeResult.length; i++) { - result[i + memcacheResult.length] = storeResult[i]; - } - } - return result; - } finally { - lock.releaseReadLock(); - } + HStoreKey key = new HStoreKey(row, column, timestamp); + HStore targetStore = stores.get(HStoreKey.extractFamily(column)); + return targetStore.get(key, numVersions); } /** @@ -1014,33 +1019,26 @@ public class HRegion implements HConstants { * checking many files needlessly. A small Bloom for each row would help us * determine which column groups are useful for that row. That would let us * avoid a bunch of disk activity. + * + * @param row + * @return Map values + * @throws IOException */ - TreeMap getFull(Text row) throws IOException { + public Map getFull(Text row) throws IOException { HStoreKey key = new HStoreKey(row, System.currentTimeMillis()); - lock.obtainReadLock(); + obtainRowLock(row); try { - TreeMap memResult = memcache.getFull(key); + TreeMap result = new TreeMap(); for (Text colFamily: stores.keySet()) { - this.stores.get(colFamily).getFull(key, memResult); + HStore targetStore = stores.get(colFamily); + targetStore.getFull(key, result); } - return memResult; + return result; } finally { - lock.releaseReadLock(); + releaseRowLock(row); } } - /** - * Get all keys matching the origin key's row/column/timestamp and those - * of an older vintage - * Default access so can be accessed out of {@link HRegionServer}. - * @param origin Where to start searching. - * @return Ordered list of keys going from newest on back. - * @throws IOException - */ - List getKeys(final HStoreKey origin) throws IOException { - return getKeys(origin, ALL_VERSIONS); - } - /** * Get versions keys matching the origin key's * row/column/timestamp and those of an older vintage @@ -1051,19 +1049,16 @@ public class HRegion implements HConstants { * @return Ordered list of versions keys going from newest back. * @throws IOException */ - List getKeys(final HStoreKey origin, final int versions) - throws IOException { - List keys = this.memcache.getKeys(origin, versions); - if (versions != ALL_VERSIONS && keys.size() >= versions) { - return keys; - } - // Check hstore for more versions. + private List getKeys(final HStoreKey origin, final int versions) + throws IOException { + + List keys = null; Text colFamily = HStoreKey.extractFamily(origin.getColumn()); HStore targetStore = stores.get(colFamily); if (targetStore != null) { // Pass versions without modification since in the store getKeys, it // includes the size of the passed keys array when counting. - keys = targetStore.getKeys(origin, keys, versions); + keys = targetStore.getKeys(origin, versions); } return keys; } @@ -1085,10 +1080,13 @@ public class HRegion implements HConstants { * @throws IOException */ public HInternalScannerInterface getScanner(Text[] cols, Text firstRow, - long timestamp, RowFilterInterface filter) - throws IOException { - lock.obtainReadLock(); + long timestamp, RowFilterInterface filter) throws IOException { + lock.readLock().lock(); try { + if (this.closed.get()) { + throw new IOException("Region " + this.getRegionName().toString() + + " closed"); + } TreeSet families = new TreeSet(); for(int i = 0; i < cols.length; i++) { families.add(HStoreKey.extractFamily(cols[i])); @@ -1101,10 +1099,10 @@ public class HRegion implements HConstants { } storelist.add(stores.get(family)); } - return new HScanner(cols, firstRow, timestamp, memcache, + return new HScanner(cols, firstRow, timestamp, storelist.toArray(new HStore [storelist.size()]), filter); } finally { - lock.releaseReadLock(); + lock.readLock().unlock(); } } @@ -1113,44 +1111,77 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// /** - * The caller wants to apply a series of writes to a single row in the - * HRegion. The caller will invoke startUpdate(), followed by a series of - * calls to put/delete, then finally either abort() or commit(). - * - *

Note that we rely on the external caller to properly abort() or - * commit() every transaction. If the caller is a network client, there - * should be a lease-system in place that automatically aborts() transactions - * after a specified quiet period. - * - * @param row Row to update - * @return lock id + * @param timestamp + * @param b * @throws IOException - * @see #put(long, Text, byte[]) */ - public long startUpdate(Text row) throws IOException { + public void batchUpdate(long timestamp, BatchUpdate b) + throws IOException { // Do a rough check that we have resources to accept a write. The check is // 'rough' in that between the resource check and the call to obtain a // read lock, resources may run out. For now, the thought is that this // will be extremely rare; we'll deal with it when it happens. checkResources(); - // Get a read lock. We will not be able to get one if we are closing or - // if this region is being split. In neither case should we be allowing - // updates. - this.lock.obtainReadLock(); - if (this.closed.get()) { - throw new IOException("Region " + this.getRegionName().toString() + - " closed"); - } + // We obtain a per-row lock, so other clients will block while one client + // performs an update. The read lock is released by the client calling + // #commit or #abort or if the HRegionServer lease on the lock expires. + // See HRegionServer#RegionListener for how the expire on HRegionServer + // invokes a HRegion#abort. + Text row = b.getRow(); + long lockid = obtainRowLock(row); + + long commitTime = + (timestamp == LATEST_TIMESTAMP) ? System.currentTimeMillis() : timestamp; + try { - // We obtain a per-row lock, so other clients will block while one client - // performs an update. The read lock is released by the client calling - // #commit or #abort or if the HRegionServer lease on the lock expires. - // See HRegionServer#RegionListener for how the expire on HRegionServer - // invokes a HRegion#abort. - return obtainRowLock(row); + List deletes = null; + for (BatchOperation op: b) { + HStoreKey key = new HStoreKey(row, op.getColumn(), commitTime); + byte[] val = null; + switch(op.getOp()) { + case PUT: + val = op.getValue(); + if (HLogEdit.isDeleted(val)) { + throw new IOException("Cannot insert value: " + val); + } + break; + + case DELETE: + if (timestamp == LATEST_TIMESTAMP) { + // Save off these deletes + if (deletes == null) { + deletes = new ArrayList(); + } + deletes.add(op.getColumn()); + } else { + val = HLogEdit.deleteBytes.get(); + } + break; + } + if (val != null) { + localput(lockid, key, val); + } + } + TreeMap edits = + this.targetColumns.remove(Long.valueOf(lockid)); + if (edits != null && edits.size() > 0) { + update(edits); + } + + if (deletes != null && deletes.size() > 0) { + // We have some LATEST_TIMESTAMP deletes to run. + for (Text column: deletes) { + deleteMultiple(row, column, LATEST_TIMESTAMP, 1); + } + } + + } catch (IOException e) { + this.targetColumns.remove(Long.valueOf(lockid)); + throw e; + } finally { - this.lock.releaseReadLock(); + releaseRowLock(row); } } @@ -1168,11 +1199,11 @@ public class HRegion implements HConstants { private synchronized void checkResources() { boolean blocked = false; - while (!checkCommitsSinceFlush()) { + while (this.memcacheSize.get() >= this.blockingMemcacheSize) { if (!blocked) { LOG.info("Blocking updates for '" + Thread.currentThread().getName() + "': Memcache size " + - StringUtils.humanReadableInt(this.memcache.getSize()) + + StringUtils.humanReadableInt(this.memcacheSize.get()) + " is >= than blocking " + StringUtils.humanReadableInt(this.blockingMemcacheSize) + " size"); } @@ -1190,45 +1221,6 @@ public class HRegion implements HConstants { } } - /* - * @return True if commits since flush is under the blocking threshold. - */ - private boolean checkCommitsSinceFlush() { - return this.memcache.getSize() < this.blockingMemcacheSize; - } - - /** - * 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 - * memory area until the user commits the change, at which point it's logged - * and placed into the memcache. - * - * This method really just tests the input, then calls an internal localput() - * method. - * - * @param lockid lock id obtained from startUpdate - * @param targetCol name of column to be updated - * @param val new value for column - * @throws IOException - */ - public void put(long lockid, Text targetCol, byte [] val) throws IOException { - if (HLogEdit.isDeleted(val)) { - throw new IOException("Cannot insert value: " + val); - } - localput(lockid, targetCol, val); - } - - /** - * Delete a value or write a value. - * This is a just a convenience method for put(). - * @param lockid lock id obtained from startUpdate - * @param targetCol name of column to be deleted - * @throws IOException - */ - public void delete(long lockid, Text targetCol) throws IOException { - localput(lockid, targetCol, HLogEdit.deleteBytes.get()); - } - /** * Delete all cells of the same age as the passed timestamp or older. * @param row @@ -1237,43 +1229,43 @@ public class HRegion implements HConstants { * @throws IOException */ public void deleteAll(final Text row, final Text column, final long ts) - throws IOException { - deleteMultiple(row, column, ts, ALL_VERSIONS); + throws IOException { + + checkColumn(column); + obtainRowLock(row); + try { + deleteMultiple(row, column, ts, ALL_VERSIONS); + } finally { + releaseRowLock(row); + } } /** * Delete one or many cells. * Used to support {@link #deleteAll(Text, Text, long)} and deletion of * latest cell. + * * @param row * @param column * @param ts Timestamp to start search on. * @param versions How many versions to delete. Pass - * {@link HConstants.ALL_VERSIONS} to delete all. + * {@link HConstants#ALL_VERSIONS} to delete all. * @throws IOException */ - void deleteMultiple(final Text row, final Text column, final long ts, - final int versions) - throws IOException { - lock.obtainReadLock(); - try { - checkColumn(column); - HStoreKey origin = new HStoreKey(row, column, ts); - synchronized(row) { - List keys = getKeys(origin, versions); - if (keys.size() > 0) { - TreeMap edits = new TreeMap(); - edits.put(column, HLogEdit.deleteBytes.get()); - for (HStoreKey key: keys) { - update(row, key.getTimestamp(), edits); - } - } + private void deleteMultiple(final Text row, final Text column, final long ts, + final int versions) throws IOException { + + HStoreKey origin = new HStoreKey(row, column, ts); + List keys = getKeys(origin, versions); + if (keys.size() > 0) { + TreeMap edits = new TreeMap(); + for (HStoreKey key: keys) { + edits.put(key, HLogEdit.deleteBytes.get()); } - } finally { - lock.releaseReadLock(); + update(edits); } } - + /** * Private implementation. * @@ -1282,146 +1274,23 @@ public class HRegion implements HConstants { * (Or until the user's write-lock expires.) * * @param lockid - * @param targetCol + * @param key * @param val Value to enter into cell * @throws IOException */ - void localput(final long lockid, final Text targetCol, - final byte [] val) - throws IOException { - checkColumn(targetCol); - - Text row = getRowFromLock(lockid); - if (row == null) { - throw new LockException("No write lock for lockid " + lockid); - } - - // This sync block makes localput() thread-safe when multiple - // threads from the same client attempt an insert on the same - // locked row (via lockid). - synchronized(row) { - // This check makes sure that another thread from the client - // hasn't aborted/committed the write-operation. - if (row != getRowFromLock(lockid)) { - throw new LockException("Locking error: put operation on lock " + - lockid + " unexpected aborted by another thread"); - } - Long lid = Long.valueOf(lockid); - TreeMap targets = this.targetColumns.get(lid); - if (targets == null) { - targets = new TreeMap(); - this.targetColumns.put(lid, targets); - } - targets.put(targetCol, val); + private void localput(final long lockid, final HStoreKey key, + final byte [] val) throws IOException { + + checkColumn(key.getColumn()); + Long lid = Long.valueOf(lockid); + TreeMap targets = this.targetColumns.get(lid); + if (targets == null) { + targets = new TreeMap(); + this.targetColumns.put(lid, targets); } + targets.put(key, val); } - /** - * Abort a pending set of writes. This dumps from memory all in-progress - * writes associated with the given row-lock. These values have not yet - * been placed in memcache or written to the log. - * - * @param lockid lock id obtained from startUpdate - * @throws IOException - */ - public void abort(long lockid) throws IOException { - Text row = getRowFromLock(lockid); - if(row == null) { - throw new LockException("No write lock for lockid " + lockid); - } - - // This sync block makes abort() thread-safe when multiple - // threads from the same client attempt to operate on the same - // locked row (via lockid). - - synchronized(row) { - - // This check makes sure another thread from the client - // hasn't aborted/committed the write-operation. - - if(row != getRowFromLock(lockid)) { - throw new LockException("Locking error: abort() operation on lock " - + lockid + " unexpected aborted by another thread"); - } - - this.targetColumns.remove(Long.valueOf(lockid)); - releaseRowLock(row); - } - } - - /** - * Commit a pending set of writes to the memcache. This also results in - * writing to the change log. - * - * Once updates hit the change log, they are safe. They will either be moved - * into an HStore in the future, or they will be recovered from the log. - * @param lockid Lock for row we're to commit. - * @param timestamp the time to associate with this change. - * @throws IOException - */ - public void commit(final long lockid, final long timestamp) - throws IOException { - // Remove the row from the pendingWrites list so - // that repeated executions won't screw this up. - Text row = getRowFromLock(lockid); - if(row == null) { - throw new LockException("No write lock for lockid " + lockid); - } - - // This check makes sure that another thread from the client - // hasn't aborted/committed the write-operation - synchronized(row) { - Long lid = Long.valueOf(lockid); - update(row, timestamp, this.targetColumns.get(lid)); - targetColumns.remove(lid); - releaseRowLock(row); - } - } - - /** - * This method for unit testing only. - * Does each operation individually so can do appropriate - * {@link HConstants#LATEST_TIMESTAMP} action. Tries to mimic how - * {@link HRegionServer#batchUpdate(Text, long, org.apache.hadoop.hbase.io.BatchUpdate)} - * works when passed a timestamp of LATEST_TIMESTAMP. - * @param lockid Lock for row we're to commit. - * @throws IOException - * @throws IOException - * @see {@link #commit(long, long)} - */ - void commit(final long lockid) throws IOException { - // Remove the row from the pendingWrites list so - // that repeated executions won't screw this up. - Text row = getRowFromLock(lockid); - if(row == null) { - throw new LockException("No write lock for lockid " + lockid); - } - - // This check makes sure that another thread from the client - // hasn't aborted/committed the write-operation - synchronized(row) { - Long lid = Long.valueOf(lockid); - TreeMap updatesByColumn = this.targetColumns.get(lid); - // Run updates one at a time so we can supply appropriate timestamp - long now = System.currentTimeMillis(); - for (Map.Entrye: updatesByColumn.entrySet()) { - if (HLogEdit.isDeleted(e.getValue())) { - // Its a delete. Delete latest. deleteMultiple calls update for us. - // Actually regets the row lock but since we already have it, should - // be fine. - deleteMultiple(row, e.getKey(), LATEST_TIMESTAMP, 1); - continue; - } - // Must be a 'put'. - TreeMap putEdit = new TreeMap(); - putEdit.put(e.getKey(), e.getValue()); - update(row, now, putEdit); - } - this.targetColumns.remove(lid); - releaseRowLock(row); - } - } - /* * Add updates first to the hlog and then add values to memcache. * Warning: Assumption is caller has lock on passed in row. @@ -1430,15 +1299,24 @@ public class HRegion implements HConstants { * @param updatesByColumn Cell updates by column * @throws IOException */ - private void update(final Text row, final long timestamp, - final TreeMap updatesByColumn) - throws IOException { + private void update(final TreeMap updatesByColumn) + throws IOException { + if (updatesByColumn == null || updatesByColumn.size() <= 0) { return; } - this.log.append(regionInfo.getRegionName(), - regionInfo.getTableDesc().getName(), row, updatesByColumn, timestamp); - this.memcache.add(row, updatesByColumn, timestamp); + synchronized (updateLock) { // prevent a cache flush + this.log.append(regionInfo.getRegionName(), + regionInfo.getTableDesc().getName(), updatesByColumn); + + for (Map.Entry e: updatesByColumn.entrySet()) { + HStoreKey key = e.getKey(); + byte[] val = e.getValue(); + this.memcacheSize.addAndGet(key.getSize() + + (val == null ? 0 : val.length)); + stores.get(HStoreKey.extractFamily(key.getColumn())).add(key, val); + } + } } ////////////////////////////////////////////////////////////////////////////// @@ -1446,7 +1324,7 @@ public class HRegion implements HConstants { ////////////////////////////////////////////////////////////////////////////// /** Make sure this is a valid row for the HRegion */ - void checkRow(Text row) throws IOException { + private void checkRow(Text row) throws IOException { if(((regionInfo.getStartKey().getLength() == 0) || (regionInfo.getStartKey().compareTo(row) <= 0)) && ((regionInfo.getEndKey().getLength() == 0) @@ -1466,7 +1344,7 @@ public class HRegion implements HConstants { * @param columnName * @throws IOException */ - void checkColumn(Text columnName) throws IOException { + private void checkColumn(Text columnName) throws IOException { Text family = new Text(HStoreKey.extractFamily(columnName) + ":"); if(! regionInfo.getTableDesc().hasFamily(family)) { throw new IOException("Requested column family " + family @@ -1495,41 +1373,50 @@ public class HRegion implements HConstants { * which maybe we'll do in the future. * * @param row Name of row to lock. + * @throws IOException * @return The id of the held lock. */ long obtainRowLock(Text row) throws IOException { checkRow(row); - synchronized(rowsToLocks) { - while(rowsToLocks.get(row) != null) { - try { - rowsToLocks.wait(); - } catch (InterruptedException ie) { - // Empty - } + lock.readLock().lock(); + try { + if (this.closed.get()) { + throw new IOException("Region " + this.getRegionName().toString() + + " closed"); } - - Long lid = Long.valueOf(Math.abs(rand.nextLong())); - rowsToLocks.put(row, lid); - locksToRows.put(lid, row); - rowsToLocks.notifyAll(); - return lid.longValue(); + synchronized (rowsToLocks) { + while (rowsToLocks.get(row) != null) { + try { + rowsToLocks.wait(); + } catch (InterruptedException ie) { + // Empty + } + } + Long lid = Long.valueOf(Math.abs(rand.nextLong())); + rowsToLocks.put(row, lid); + locksToRows.put(lid, row); + rowsToLocks.notifyAll(); + return lid.longValue(); + } + } finally { + lock.readLock().unlock(); } } Text getRowFromLock(long lockid) { // Pattern is that all access to rowsToLocks and/or to // locksToRows is via a lock on rowsToLocks. - synchronized(rowsToLocks) { + synchronized (rowsToLocks) { return locksToRows.get(Long.valueOf(lockid)); } } /** * Release the row lock! - * @param lock Name of row whose lock we are to release + * @param row Name of row whose lock we are to release */ void releaseRowLock(Text row) { - synchronized(rowsToLocks) { + synchronized (rowsToLocks) { long lockid = rowsToLocks.remove(row).longValue(); locksToRows.remove(Long.valueOf(lockid)); rowsToLocks.notifyAll(); @@ -1537,7 +1424,7 @@ public class HRegion implements HConstants { } private void waitOnRowLocks() { - synchronized (this.rowsToLocks) { + synchronized (rowsToLocks) { while (this.rowsToLocks.size() > 0) { try { this.rowsToLocks.wait(); @@ -1557,51 +1444,33 @@ public class HRegion implements HConstants { /** * HScanner is an iterator through a bunch of rows in an HRegion. */ - private static class HScanner implements HInternalScannerInterface { + private class HScanner implements HInternalScannerInterface { private HInternalScannerInterface[] scanners; - private TreeMap[] resultSets; - private HStoreKey[] keys; private boolean wildcardMatch = false; private boolean multipleMatchers = false; - private RowFilterInterface dataFilter; /** Create an HScanner with a handle on many HStores. */ @SuppressWarnings("unchecked") - HScanner(Text[] cols, Text firstRow, long timestamp, HMemcache memcache, - HStore[] stores, RowFilterInterface filter) throws IOException { - this.dataFilter = filter; - if (null != dataFilter) { - dataFilter.reset(); - } - this.scanners = new HInternalScannerInterface[stores.length + 1]; - this.resultSets = new TreeMap[scanners.length]; - this.keys = new HStoreKey[scanners.length]; + HScanner(Text[] cols, Text firstRow, long timestamp, HStore[] stores, + RowFilterInterface filter) throws IOException { + this.scanners = new HInternalScannerInterface[stores.length]; - // Advance to the first key in each store. - // All results will match the required column-set and scanTime. +// Advance to the first key in each store. +// All results will match the required column-set and scanTime. - // NOTE: the memcache scanner should be the first scanner try { - HInternalScannerInterface scanner = - memcache.getScanner(timestamp, cols, firstRow); - if (scanner.isWildcardScanner()) { - this.wildcardMatch = true; - } - if (scanner.isMultipleMatchScanner()) { - this.multipleMatchers = true; - } - scanners[0] = scanner; - for (int i = 0; i < stores.length; i++) { - scanner = stores[i].getScanner(timestamp, cols, firstRow); - if (scanner.isWildcardScanner()) { - this.wildcardMatch = true; + HInternalScannerInterface scanner = + scanners[i] = + stores[i].getScanner(timestamp, cols, firstRow, filter); + + if (scanner.isWildcardScanner()) { + this.wildcardMatch = true; + } + if (scanner.isMultipleMatchScanner()) { + this.multipleMatchers = true; + } } - if (scanner.isMultipleMatchScanner()) { - this.multipleMatchers = true; - } - scanners[i + 1] = scanner; - } } catch(IOException e) { for (int i = 0; i < this.scanners.length; i++) { @@ -1611,13 +1480,9 @@ public class HRegion implements HConstants { } throw e; } - for (int i = 0; i < scanners.length; i++) { - keys[i] = new HStoreKey(); - resultSets[i] = new TreeMap(); - if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { - closeScanner(i); - } - } + // As we have now successfully completed initialization, increment the + // activeScanner count. + activeScannerCount.incrementAndGet(); } /** @return true if the scanner is a wild card scanner */ @@ -1633,152 +1498,17 @@ public class HRegion implements HConstants { /** {@inheritDoc} */ public boolean next(HStoreKey key, SortedMap results) throws IOException { - // Filtered flag is set by filters. If a cell has been 'filtered out' - // -- i.e. it is not to be returned to the caller -- the flag is 'true'. - boolean filtered = true; - boolean moreToFollow = true; - while (filtered && moreToFollow) { - // Find the lowest-possible key. - Text chosenRow = null; - long chosenTimestamp = -1; - for (int i = 0; i < this.keys.length; i++) { - if (scanners[i] != null && - (chosenRow == null || - (keys[i].getRow().compareTo(chosenRow) < 0) || - ((keys[i].getRow().compareTo(chosenRow) == 0) && - (keys[i].getTimestamp() > chosenTimestamp)))) { - chosenRow = new Text(keys[i].getRow()); - chosenTimestamp = keys[i].getTimestamp(); - } - } - - // Filter whole row by row key? - filtered = dataFilter != null? dataFilter.filter(chosenRow) : false; - - // Store the key and results for each sub-scanner. Merge them as - // appropriate. - if (chosenTimestamp >= 0 && !filtered) { - // Here we are setting the passed in key with current row+timestamp - key.setRow(chosenRow); - key.setVersion(chosenTimestamp); - key.setColumn(HConstants.EMPTY_TEXT); - // Keep list of deleted cell keys within this row. We need this - // because as we go through scanners, the delete record may be in an - // early scanner and then the same record with a non-delete, non-null - // value in a later. Without history of what we've seen, we'll return - // deleted values. This List should not ever grow too large since we - // are only keeping rows and columns that match those set on the - // scanner and which have delete values. If memory usage becomes a - // problem, could redo as bloom filter. - List deletes = new ArrayList(); - for (int i = 0; i < scanners.length && !filtered; i++) { - while ((scanners[i] != null - && !filtered - && moreToFollow) - && (keys[i].getRow().compareTo(chosenRow) == 0)) { - // If we are doing a wild card match or there are multiple - // matchers per column, we need to scan all the older versions of - // this row to pick up the rest of the family members - if (!wildcardMatch - && !multipleMatchers - && (keys[i].getTimestamp() != chosenTimestamp)) { - break; - } - - // Filter out null criteria columns that are not null - if (dataFilter != null) { - filtered = dataFilter.filterNotNull(resultSets[i]); - } - - // NOTE: We used to do results.putAll(resultSets[i]); - // but this had the effect of overwriting newer - // values with older ones. So now we only insert - // a result if the map does not contain the key. - HStoreKey hsk = new HStoreKey(key.getRow(), EMPTY_TEXT, - key.getTimestamp()); - for (Map.Entry e : resultSets[i].entrySet()) { - hsk.setColumn(e.getKey()); - if (HLogEdit.isDeleted(e.getValue())) { - if (!deletes.contains(hsk)) { - // Key changes as we cycle the for loop so add a copy to - // the set of deletes. - deletes.add(new HStoreKey(hsk)); - } - } else if (!deletes.contains(hsk) && - !filtered && - moreToFollow && - !results.containsKey(e.getKey())) { - if (dataFilter != null) { - // Filter whole row by column data? - filtered = - dataFilter.filter(chosenRow, e.getKey(), e.getValue()); - if (filtered) { - results.clear(); - break; - } - } - results.put(e.getKey(), e.getValue()); - } - } - resultSets[i].clear(); - if (!scanners[i].next(keys[i], resultSets[i])) { - closeScanner(i); - } - } - } - } - - for (int i = 0; i < scanners.length; i++) { - // If the current scanner is non-null AND has a lower-or-equal - // row label, then its timestamp is bad. We need to advance it. - while ((scanners[i] != null) && - (keys[i].getRow().compareTo(chosenRow) <= 0)) { - resultSets[i].clear(); - if (!scanners[i].next(keys[i], resultSets[i])) { - closeScanner(i); - } - } - } - - moreToFollow = chosenTimestamp >= 0; - - if (dataFilter != null) { - if (moreToFollow) { - dataFilter.rowProcessed(filtered, chosenRow); - } - if (dataFilter.filterAllRemaining()) { - moreToFollow = false; - LOG.debug("page limit"); - } - } - if (LOG.isDebugEnabled()) { - if (this.dataFilter != null) { - LOG.debug("ROWKEY = " + chosenRow + ", FILTERED = " + filtered); - } - } - - if (results.size() <= 0 && !filtered) { - // There were no results found for this row. Marked it as - // 'filtered'-out otherwise we will not move on to the next row. - filtered = true; - } - } - - // If we got no results, then there is no more to follow. - if (results == null || results.size() <= 0) { - moreToFollow = false; - } - - // Make sure scanners closed if no more results - if (!moreToFollow) { - for (int i = 0; i < scanners.length; i++) { - if (null != scanners[i]) { + boolean haveResults = false; + for (int i = 0; i < scanners.length; i++) { + if (scanners[i] != null) { + if (scanners[i].next(key, results)) { + haveResults = true; + } else { closeScanner(i); } } } - - return moreToFollow; + return haveResults; } @@ -1792,8 +1522,6 @@ public class HRegion implements HConstants { } } finally { scanners[i] = null; - keys[i] = null; - resultSets[i] = null; } } @@ -1801,13 +1529,29 @@ public class HRegion implements HConstants { * {@inheritDoc} */ public void close() { - for(int i = 0; i < scanners.length; i++) { - if(scanners[i] != null) { - closeScanner(i); + try { + for(int i = 0; i < scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + } finally { + synchronized (activeScannerCount) { + int scanners = activeScannerCount.decrementAndGet(); + if (scanners < 0) { + LOG.error("active scanner count less than zero: " + scanners + + " resetting to zero"); + activeScannerCount.set(0); + scanners = 0; + } + if (scanners == 0) { + activeScannerCount.notifyAll(); + } } } } + /** {@inheritDoc} */ public Iterator>> iterator() { throw new UnsupportedOperationException("Unimplemented serverside. " + "next(HStoreKey, StortedMap(...) is more efficient"); @@ -1852,12 +1596,21 @@ public class HRegion implements HConstants { * * @throws IOException */ - static void addRegionToMETA(HRegion meta, HRegion r) - throws IOException { + static void addRegionToMETA(HRegion meta, HRegion r) throws IOException { + meta.checkResources(); // The row key is the region name - long writeid = meta.startUpdate(r.getRegionName()); - meta.put(writeid, COL_REGIONINFO, Writables.getBytes(r.getRegionInfo())); - meta.commit(writeid, System.currentTimeMillis()); + Text row = r.getRegionName(); + meta.obtainRowLock(row); + try { + HStoreKey key = + new HStoreKey(row, COL_REGIONINFO, System.currentTimeMillis()); + TreeMap edits = new TreeMap(); + edits.put(key, Writables.getBytes(r.getRegionInfo())); + meta.update(edits); + + } finally { + meta.releaseRowLock(row); + } } /** diff --git a/src/java/org/apache/hadoop/hbase/HRegionServer.java b/src/java/org/apache/hadoop/hbase/HRegionServer.java index 952bc0c1334..63eaaab444d 100644 --- a/src/java/org/apache/hadoop/hbase/HRegionServer.java +++ b/src/java/org/apache/hadoop/hbase/HRegionServer.java @@ -47,7 +47,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.filter.RowFilterInterface; -import org.apache.hadoop.hbase.io.BatchOperation; import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.util.FSUtils; @@ -194,15 +193,12 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { List nonClosedRegionsToCheck = getRegionsToCheck(); for(HRegion cur: nonClosedRegionsToCheck) { try { - if (cur.needsCompaction()) { - cur.compactStores(); - } - // After compaction, it probably needs splitting. May also need - // splitting just because one of the memcache flushes was big. - Text midKey = new Text(); - if (cur.needsSplit(midKey)) { - split(cur, midKey); + if (cur.compactIfNeeded()) { + // After compaction, it probably needs splitting. May also need + // splitting just because one of the memcache flushes was big. + split(cur); } + } catch(IOException e) { //TODO: What happens if this fails? Are we toast? LOG.error("Split or compaction failed", e); @@ -213,10 +209,13 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { } } - private void split(final HRegion region, final Text midKey) - throws IOException { + private void split(final HRegion region) throws IOException { final HRegionInfo oldRegionInfo = region.getRegionInfo(); - final HRegion[] newRegions = region.closeAndSplit(midKey, this); + final HRegion[] newRegions = region.splitRegion(this); + + if (newRegions == null) { + return; // Didn't need to be split + } // When a region is split, the META table needs to updated if we're // splitting a 'normal' region, and the ROOT table needs to be @@ -302,7 +301,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // Flush them, if necessary for(HRegion cur: nonClosedRegionsToFlush) { try { - cur.optionallyFlush(); + cur.flushcache(); } catch (DroppedSnapshotException e) { // Cache flush can fail in a few places. If it fails in a critical // section, we get a DroppedSnapshotException and a replay of hlog @@ -1046,7 +1045,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { try { HRegion region = getRegion(regionName); MapWritable result = new MapWritable(); - TreeMap map = region.getFull(row); + Map map = region.getFull(row); for (Map.Entry es: map.entrySet()) { result.put(new HStoreKey(row, es.getKey()), new ImmutableBytesWritable(es.getValue())); @@ -1100,46 +1099,13 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { /** {@inheritDoc} */ public void batchUpdate(Text regionName, long timestamp, BatchUpdate b) - throws IOException { + throws IOException { + checkOpen(); this.requestCount.incrementAndGet(); - // If timestamp == LATEST_TIMESTAMP and we have deletes, then they need - // special treatment. For these we need to first find the latest cell so - // when we write the delete, we write it with the latest cells' timestamp - // so the delete record overshadows. This means deletes and puts do not - // happen within the same row lock. - List deletes = null; + HRegion region = getRegion(regionName); try { - long lockid = startUpdate(regionName, b.getRow()); - for (BatchOperation op: b) { - switch(op.getOp()) { - case PUT: - put(regionName, lockid, op.getColumn(), op.getValue()); - break; - - case DELETE: - if (timestamp == LATEST_TIMESTAMP) { - // Save off these deletes. - if (deletes == null) { - deletes = new ArrayList(); - } - deletes.add(op.getColumn()); - } else { - delete(regionName, lockid, op.getColumn()); - } - break; - } - } - commit(regionName, lockid, - (timestamp == LATEST_TIMESTAMP)? System.currentTimeMillis(): timestamp); - - if (deletes != null && deletes.size() > 0) { - // We have some LATEST_TIMESTAMP deletes to run. - HRegion r = getRegion(regionName); - for (Text column: deletes) { - r.deleteMultiple(b.getRow(), column, LATEST_TIMESTAMP, 1); - } - } + region.batchUpdate(timestamp, b); } catch (IOException e) { checkFileSystem(); throw e; @@ -1234,24 +1200,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { // Methods that do the actual work for the remote API // - protected long startUpdate(Text regionName, Text row) throws IOException { - HRegion region = getRegion(regionName); - return region.startUpdate(row); - } - - protected void put(final Text regionName, final long lockid, - final Text column, final byte [] val) - throws IOException { - HRegion region = getRegion(regionName, true); - region.put(lockid, column, val); - } - - protected void delete(Text regionName, long lockid, Text column) - throws IOException { - HRegion region = getRegion(regionName); - region.delete(lockid, column); - } - /** {@inheritDoc} */ public void deleteAll(final Text regionName, final Text row, final Text column, final long timestamp) @@ -1260,13 +1208,6 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { region.deleteAll(row, column, timestamp); } - protected void commit(Text regionName, final long lockid, - final long timestamp) throws IOException { - - HRegion region = getRegion(regionName, true); - region.commit(lockid, timestamp); - } - /** * @return Info on this server. */ @@ -1379,6 +1320,7 @@ public class HRegionServer implements HConstants, HRegionInterface, Runnable { */ protected List getRegionsToCheck() { ArrayList regionsToCheck = new ArrayList(); + //TODO: is this locking necessary? lock.readLock().lock(); try { regionsToCheck.addAll(this.onlineRegions.values()); diff --git a/src/java/org/apache/hadoop/hbase/HStore.java b/src/java/org/apache/hadoop/hbase/HStore.java index 621dbf9a80d..7be2737e38f 100644 --- a/src/java/org/apache/hadoop/hbase/HStore.java +++ b/src/java/org/apache/hadoop/hbase/HStore.java @@ -34,6 +34,8 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.Vector; import java.util.Map.Entry; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -41,6 +43,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.filter.RowFilterInterface; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.io.MapFile; import org.apache.hadoop.io.SequenceFile; @@ -64,11 +67,384 @@ import org.onelab.filter.RetouchedBloomFilter; class HStore implements HConstants { static final Log LOG = LogFactory.getLog(HStore.class); + /** + * The Memcache holds in-memory modifications to the HRegion. This is really a + * wrapper around a TreeMap that helps us when staging the Memcache out to disk. + */ + static class Memcache { + + // Note that since these structures are always accessed with a lock held, + // no additional synchronization is required. + + @SuppressWarnings("hiding") + private final SortedMap memcache = + Collections.synchronizedSortedMap(new TreeMap()); + + volatile SortedMap snapshot; + + @SuppressWarnings("hiding") + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + + /** + * Constructor + */ + Memcache() { + snapshot = + Collections.synchronizedSortedMap(new TreeMap()); + } + + /** + * Creates a snapshot of the current Memcache + */ + void snapshot() { + this.lock.writeLock().lock(); + try { + if (memcache.size() != 0) { + snapshot.putAll(memcache); + memcache.clear(); + } + } finally { + this.lock.writeLock().unlock(); + } + } + + /** + * @return memcache snapshot + */ + SortedMap getSnapshot() { + this.lock.writeLock().lock(); + try { + SortedMap currentSnapshot = snapshot; + snapshot = + Collections.synchronizedSortedMap(new TreeMap()); + + return currentSnapshot; + + } finally { + this.lock.writeLock().unlock(); + } + } + + /** + * Store a value. + * @param key + * @param value + */ + void add(final HStoreKey key, final byte[] value) { + this.lock.readLock().lock(); + try { + memcache.put(key, value); + + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Look back through all the backlog TreeMaps to find the target. + * @param key + * @param numVersions + * @return An array of byte arrays ordered by timestamp. + */ + List get(final HStoreKey key, final int numVersions) { + this.lock.readLock().lock(); + try { + ArrayList results = internalGet(memcache, key, numVersions); + results.addAll(results.size(), + internalGet(snapshot, key, numVersions - results.size())); + return results; + + } finally { + this.lock.readLock().unlock(); + } + } + + /** + * Return all the available columns for the given key. The key indicates a + * row and timestamp, but not a column name. + * + * The returned object should map column names to byte arrays (byte[]). + * @param key + * @param results + */ + void getFull(HStoreKey key, SortedMap results) { + this.lock.readLock().lock(); + try { + internalGetFull(memcache, key, results); + internalGetFull(snapshot, key, results); + + } finally { + this.lock.readLock().unlock(); + } + } + + private void internalGetFull(SortedMap map, HStoreKey key, + SortedMap results) { + + SortedMap tailMap = map.tailMap(key); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey itKey = es.getKey(); + Text itCol = itKey.getColumn(); + if (results.get(itCol) == null && key.matchesWithoutColumn(itKey)) { + byte [] val = tailMap.get(itKey); + results.put(itCol, val); + + } else if (key.getRow().compareTo(itKey.getRow()) < 0) { + break; + } + } + } + + /** + * Examine a single map for the desired key. + * + * TODO - This is kinda slow. We need a data structure that allows for + * proximity-searches, not just precise-matches. + * + * @param map + * @param key + * @param numVersions + * @return Ordered list of items found in passed map. If no + * matching values, returns an empty list (does not return null). + */ + private ArrayList internalGet( + final SortedMap map, final HStoreKey key, + final int numVersions) { + + ArrayList result = new ArrayList(); + // TODO: If get is of a particular version -- numVersions == 1 -- we + // should be able to avoid all of the tailmap creations and iterations + // below. + HStoreKey curKey = new HStoreKey(key); + SortedMap tailMap = map.tailMap(curKey); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey itKey = es.getKey(); + if (itKey.matchesRowCol(curKey)) { + if (!HLogEdit.isDeleted(es.getValue())) { + result.add(tailMap.get(itKey)); + curKey.setVersion(itKey.getTimestamp() - 1); + } + } + if (numVersions > 0 && result.size() >= numVersions) { + break; + } + } + return result; + } + + /** + * Get versions keys matching the origin key's + * row/column/timestamp and those of an older vintage + * Default access so can be accessed out of {@link HRegionServer}. + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. + * @return Ordered list of versions keys going from newest back. + * @throws IOException + */ + List getKeys(final HStoreKey origin, final int versions) { + this.lock.readLock().lock(); + try { + List results = + internalGetKeys(this.memcache, origin, versions); + results.addAll(results.size(), internalGetKeys(snapshot, origin, + versions == HConstants.ALL_VERSIONS ? versions : + (versions - results.size()))); + return results; + + } finally { + this.lock.readLock().unlock(); + } + } + + /* + * @param origin Where to start searching. + * @param versions How many versions to return. Pass + * {@link HConstants.ALL_VERSIONS} to retrieve all. + * @return List of all keys that are of the same row and column and of + * equal or older timestamp. If no keys, returns an empty List. Does not + * return null. + */ + private List internalGetKeys(final SortedMap map, + final HStoreKey origin, final int versions) { + + List result = new ArrayList(); + SortedMap tailMap = map.tailMap(origin); + for (Map.Entry es: tailMap.entrySet()) { + HStoreKey key = es.getKey(); + if (!key.matchesRowCol(origin)) { + break; + } + if (!HLogEdit.isDeleted(es.getValue())) { + result.add(key); + if (versions != HConstants.ALL_VERSIONS && result.size() >= versions) { + // We have enough results. Return. + break; + } + } + } + return result; + } + + + /** + * @param key + * @return True if an entry and its content is {@link HGlobals.deleteBytes}. + * Use checking values in store. On occasion the memcache has the fact that + * the cell has been deleted. + */ + boolean isDeleted(final HStoreKey key) { + return HLogEdit.isDeleted(this.memcache.get(key)); + } + + /** + * @return a scanner over the keys in the Memcache + */ + HInternalScannerInterface getScanner(long timestamp, + Text targetCols[], Text firstRow) throws IOException { + + // Here we rely on ReentrantReadWriteLock's ability to acquire multiple + // locks by the same thread and to be able to downgrade a write lock to + // a read lock. We need to hold a lock throughout this method, but only + // need the write lock while creating the memcache snapshot + + this.lock.writeLock().lock(); // hold write lock during memcache snapshot + snapshot(); // snapshot memcache + this.lock.readLock().lock(); // acquire read lock + this.lock.writeLock().unlock(); // downgrade to read lock + try { + // Prevent a cache flush while we are constructing the scanner + + return new MemcacheScanner(timestamp, targetCols, firstRow); + + } finally { + this.lock.readLock().unlock(); + } + } + + ////////////////////////////////////////////////////////////////////////////// + // MemcacheScanner implements the HScannerInterface. + // It lets the caller scan the contents of the Memcache. + ////////////////////////////////////////////////////////////////////////////// + + class MemcacheScanner extends HAbstractScanner { + SortedMap backingMap; + Iterator keyIterator; + + @SuppressWarnings("unchecked") + MemcacheScanner(final long timestamp, final Text targetCols[], + final Text firstRow) throws IOException { + + super(timestamp, targetCols); + try { + this.backingMap = new TreeMap(); + this.backingMap.putAll(snapshot); + this.keys = new HStoreKey[1]; + this.vals = new byte[1][]; + + // Generate list of iterators + + HStoreKey firstKey = new HStoreKey(firstRow); + if (firstRow != null && firstRow.getLength() != 0) { + keyIterator = + backingMap.tailMap(firstKey).keySet().iterator(); + + } else { + keyIterator = backingMap.keySet().iterator(); + } + + while (getNext(0)) { + if (!findFirstRow(0, firstRow)) { + continue; + } + if (columnMatch(0)) { + break; + } + } + } catch (RuntimeException ex) { + LOG.error("error initializing Memcache scanner: ", ex); + close(); + IOException e = new IOException("error initializing Memcache scanner"); + e.initCause(ex); + throw e; + + } catch(IOException ex) { + LOG.error("error initializing Memcache scanner: ", ex); + close(); + throw ex; + } + } + + /** + * The user didn't want to start scanning at the first row. This method + * seeks to the requested row. + * + * @param i which iterator to advance + * @param firstRow seek to this row + * @return true if this is the first row + */ + @Override + boolean findFirstRow(int i, Text firstRow) { + return firstRow.getLength() == 0 || + keys[i].getRow().compareTo(firstRow) >= 0; + } + + /** + * Get the next value from the specified iterator. + * + * @param i Which iterator to fetch next value from + * @return true if there is more data available + */ + @Override + boolean getNext(int i) { + boolean result = false; + while (true) { + if (!keyIterator.hasNext()) { + closeSubScanner(i); + break; + } + // Check key is < than passed timestamp for this scanner. + HStoreKey hsk = keyIterator.next(); + if (hsk == null) { + throw new NullPointerException("Unexpected null key"); + } + if (hsk.getTimestamp() <= this.timestamp) { + this.keys[i] = hsk; + this.vals[i] = backingMap.get(keys[i]); + result = true; + break; + } + } + return result; + } + + /** Shut down an individual map iterator. */ + @Override + void closeSubScanner(int i) { + keyIterator = null; + keys[i] = null; + vals[i] = null; + backingMap = null; + } + + /** Shut down map iterators */ + public void close() { + if (!scannerClosed) { + if(keyIterator != null) { + closeSubScanner(0); + } + scannerClosed = true; + } + } + } + } + static final String COMPACTION_TO_REPLACE = "toreplace"; static final String COMPACTION_DONE = "done"; private static final String BLOOMFILTER_FILE_NAME = "filter"; + final Memcache memcache = new Memcache(); Path dir; Text regionName; String encodedRegionName; @@ -87,12 +463,14 @@ class HStore implements HConstants { Integer compactLock = new Integer(0); Integer flushLock = new Integer(0); - final HLocking lock = new HLocking(); + final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + final AtomicInteger activeScanners = new AtomicInteger(0); /* Sorted Map of readers keyed by sequence id (Most recent should be last in * in list). */ - TreeMap storefiles = new TreeMap(); + SortedMap storefiles = + Collections.synchronizedSortedMap(new TreeMap()); /* Sorted Map of readers keyed by sequence id (Most recent should be last in * in list). @@ -101,9 +479,12 @@ class HStore implements HConstants { Random rand = new Random(); - private long maxSeqId; + private volatile long maxSeqId; - private int compactionThreshold; + private final int compactionThreshold; + + private final ReentrantReadWriteLock newScannerLock = + new ReentrantReadWriteLock(); /** * An HStore is a set of zero or more MapFiles, which stretch backwards over @@ -140,8 +521,8 @@ class HStore implements HConstants { */ HStore(Path dir, Text regionName, String encodedName, HColumnDescriptor family, FileSystem fs, Path reconstructionLog, - HBaseConfiguration conf) - throws IOException { + HBaseConfiguration conf) throws IOException { + this.dir = dir; this.compactionDir = new Path(dir, "compaction.dir"); this.regionName = regionName; @@ -255,8 +636,8 @@ class HStore implements HConstants { * reflected in the MapFiles.) */ private void doReconstructionLog(final Path reconstructionLog, - final long maxSeqID) - throws UnsupportedEncodingException, IOException { + final long maxSeqID) throws UnsupportedEncodingException, IOException { + if (reconstructionLog == null || !fs.exists(reconstructionLog)) { // Nothing to do. return; @@ -264,8 +645,10 @@ class HStore implements HConstants { long maxSeqIdInLog = -1; TreeMap reconstructedCache = new TreeMap(); - SequenceFile.Reader login = - new SequenceFile.Reader(this.fs, reconstructionLog, this.conf); + + SequenceFile.Reader login = new SequenceFile.Reader(this.fs, + reconstructionLog, this.conf); + try { HLogKey key = new HLogKey(); HLogEdit val = new HLogEdit(); @@ -310,7 +693,7 @@ class HStore implements HConstants { if (LOG.isDebugEnabled()) { LOG.debug("flushing reconstructionCache"); } - flushCacheHelper(reconstructedCache, maxSeqIdInLog + 1, true); + internalFlushCache(reconstructedCache, maxSeqIdInLog + 1); } } @@ -402,24 +785,44 @@ class HStore implements HConstants { // End bloom filters ////////////////////////////////////////////////////////////////////////////// + /** + * Adds a value to the memcache + * + * @param key + * @param value + */ + void add(HStoreKey key, byte[] value) { + lock.readLock().lock(); + try { + this.memcache.add(key, value); + + } finally { + lock.readLock().unlock(); + } + } + /** * Close all the MapFile readers + * + * We don't need to worry about subsequent requests because the HRegion holds + * a write lock that will prevent any more reads or writes. + * * @throws IOException */ - Vector close() throws IOException { - Vector result = null; - this.lock.obtainWriteLock(); + List close() throws IOException { + ArrayList result = null; + this.lock.writeLock().lock(); try { for (MapFile.Reader reader: this.readers.values()) { reader.close(); } this.readers.clear(); - result = new Vector(storefiles.values()); + result = new ArrayList(storefiles.values()); this.storefiles.clear(); LOG.debug("closed " + this.storeName); return result; } finally { - this.lock.releaseWriteLock(); + this.lock.writeLock().unlock(); } } @@ -428,6 +831,14 @@ class HStore implements HConstants { // Flush changes to disk ////////////////////////////////////////////////////////////////////////////// + /** + * Prior to doing a cache flush, we need to snapshot the memcache. Locking is + * handled by the memcache. + */ + void snapshotMemcache() { + this.memcache.snapshot(); + } + /** * Write out a brand-new set of items to the disk. * @@ -438,21 +849,18 @@ class HStore implements HConstants { * * Return the entire list of HStoreFiles currently used by the HStore. * - * @param inputCache memcache to flush * @param logCacheFlushId flush sequence number * @throws IOException */ - void flushCache(final SortedMap inputCache, - final long logCacheFlushId) - throws IOException { - flushCacheHelper(inputCache, logCacheFlushId, true); + void flushCache(final long logCacheFlushId) throws IOException { + internalFlushCache(memcache.getSnapshot(), logCacheFlushId); } - void flushCacheHelper(SortedMap inputCache, - long logCacheFlushId, boolean addToAvailableMaps) - throws IOException { + private void internalFlushCache(SortedMap cache, + long logCacheFlushId) throws IOException { + synchronized(flushLock) { - // A. Write the TreeMap out to the disk + // A. Write the Maps out to the disk HStoreFile flushedFile = HStoreFile.obtainNewHStoreFile(conf, dir, encodedRegionName, familyName, fs); String name = flushedFile.toString(); @@ -471,10 +879,11 @@ class HStore implements HConstants { // Related, looks like 'merging compactions' in BigTable paper interlaces // a memcache flush. We don't. try { - for (Map.Entry es: inputCache.entrySet()) { + for (Map.Entry es: cache.entrySet()) { HStoreKey curkey = es.getKey(); - if (this.familyName. - equals(HStoreKey.extractFamily(curkey.getColumn()))) { + if (this.familyName.equals(HStoreKey.extractFamily( + curkey.getColumn()))) { + out.append(curkey, new ImmutableBytesWritable(es.getValue())); } } @@ -487,44 +896,30 @@ class HStore implements HConstants { flushedFile.writeInfo(fs, logCacheFlushId); // C. Flush the bloom filter if any - if(bloomFilter != null) { + if (bloomFilter != null) { flushBloomFilter(); } // D. Finally, make the new MapFile available. - if(addToAvailableMaps) { - this.lock.obtainWriteLock(); - try { - Long flushid = Long.valueOf(logCacheFlushId); - // Open the map file reader. - this.readers.put(flushid, + this.lock.writeLock().lock(); + try { + Long flushid = Long.valueOf(logCacheFlushId); + // Open the map file reader. + this.readers.put(flushid, flushedFile.getReader(this.fs, this.bloomFilter)); - this.storefiles.put(flushid, flushedFile); - if(LOG.isDebugEnabled()) { - LOG.debug("Added " + name + - " with sequence id " + logCacheFlushId + " and size " + + this.storefiles.put(flushid, flushedFile); + if(LOG.isDebugEnabled()) { + LOG.debug("Added " + name + + " with sequence id " + logCacheFlushId + " and size " + StringUtils.humanReadableInt(flushedFile.length())); - } - } finally { - this.lock.releaseWriteLock(); } + } finally { + this.lock.writeLock().unlock(); } return; } } - /** - * @return - vector of all the HStore files in use - */ - Vector getAllStoreFiles() { - this.lock.obtainReadLock(); - try { - return new Vector(storefiles.values()); - } finally { - this.lock.releaseReadLock(); - } - } - ////////////////////////////////////////////////////////////////////////////// // Compaction ////////////////////////////////////////////////////////////////////////////// @@ -532,9 +927,16 @@ class HStore implements HConstants { /** * @return True if this store needs compaction. */ - public boolean needsCompaction() { - return this.storefiles != null && - this.storefiles.size() >= this.compactionThreshold; + boolean needsCompaction() { + boolean compactionNeeded = false; + if (this.storefiles != null) { + compactionNeeded = this.storefiles.size() >= this.compactionThreshold; + if (LOG.isDebugEnabled()) { + LOG.debug("compaction for HStore " + regionName + "/" + familyName + + (compactionNeeded ? " " : " not ") + "needed."); + } + } + return compactionNeeded; } /** @@ -542,44 +944,26 @@ class HStore implements HConstants { * thread must be able to block for long periods. * *

During this time, the HStore can work as usual, getting values from - * MapFiles and writing new MapFiles from given memcaches. + * MapFiles and writing new MapFiles from the Memcache. * * Existing MapFiles are not destroyed until the new compacted TreeMap is * completely written-out to disk. * - * The compactLock block prevents multiple simultaneous compactions. + * The compactLock prevents multiple simultaneous compactions. * The structureLock prevents us from interfering with other write operations. * * We don't want to hold the structureLock for the whole time, as a compact() * can be lengthy and we want to allow cache-flushes during this period. * @throws IOException + * + * @return true if compaction completed successfully */ - void compact() throws IOException { - compactHelper(false); - } - - void compactHelper(final boolean deleteSequenceInfo) - throws IOException { - compactHelper(deleteSequenceInfo, -1); - } - - /* - * @param deleteSequenceInfo True if we are to set the sequence number to -1 - * on compacted file. - * @param maxSeenSeqID We may have already calculated the maxSeenSeqID. If - * so, pass it here. Otherwise, pass -1 and it will be calculated inside in - * this method. - * @param deleteSequenceInfo - * @param maxSeenSeqID - * @throws IOException - */ - void compactHelper(final boolean deleteSequenceInfo, long maxSeenSeqID) - throws IOException { - long maxId = maxSeenSeqID; - synchronized(compactLock) { + boolean compact() throws IOException { + long maxId = -1; + synchronized (compactLock) { Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir, encodedRegionName, familyName); - if(LOG.isDebugEnabled()) { + if (LOG.isDebugEnabled()) { LOG.debug("started compaction of " + storefiles.size() + " files in " + curCompactStore.toString()); } @@ -591,19 +975,22 @@ class HStore implements HConstants { } } try { - List toCompactFiles = getFilesToCompact(); + // Storefiles are keyed by sequence id. The oldest file comes first. + // We need to return out of here a List that has the newest file as + // first. + List filesToCompact = + new ArrayList(this.storefiles.values()); + Collections.reverse(filesToCompact); + HStoreFile compactedOutputFile = new HStoreFile(conf, this.compactionDir, encodedRegionName, familyName, -1); - if (toCompactFiles.size() < 1 || - (toCompactFiles.size() == 1 && - !toCompactFiles.get(0).isReference())) { + if (filesToCompact.size() < 1 || + (filesToCompact.size() == 1 && + !filesToCompact.get(0).isReference())) { if (LOG.isDebugEnabled()) { LOG.debug("nothing to compact for " + this.storeName); } - if (deleteSequenceInfo && toCompactFiles.size() == 1) { - toCompactFiles.get(0).writeInfo(fs, -1); - } - return; + return false; } if (!fs.mkdirs(curCompactStore)) { @@ -613,10 +1000,10 @@ class HStore implements HConstants { // Compute the max-sequenceID seen in any of the to-be-compacted // TreeMaps if it hasn't been passed in to us. if (maxId == -1) { - for (HStoreFile hsf: toCompactFiles) { + for (HStoreFile hsf: filesToCompact) { long seqid = hsf.loadInfo(fs); - if(seqid > 0) { - if(seqid > maxId) { + if (seqid > 0) { + if (seqid > maxId) { maxId = seqid; } } @@ -624,17 +1011,16 @@ class HStore implements HConstants { } // Step through them, writing to the brand-new TreeMap - MapFile.Writer compactedOut = - compactedOutputFile.getWriter(this.fs, this.compression, - this.bloomFilter); + MapFile.Writer compactedOut = compactedOutputFile.getWriter(this.fs, + this.compression, this.bloomFilter); try { - compact(compactedOut, toCompactFiles); + compactHStoreFiles(compactedOut, filesToCompact); } finally { compactedOut.close(); } // Now, write out an HSTORE_LOGINFOFILE for the brand-new TreeMap. - if((! deleteSequenceInfo) && maxId >= 0) { + if (maxId >= 0) { compactedOutputFile.writeInfo(fs, maxId); } else { compactedOutputFile.writeInfo(fs, -1); @@ -644,8 +1030,8 @@ class HStore implements HConstants { Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE); DataOutputStream out = new DataOutputStream(fs.create(filesToReplace)); try { - out.writeInt(toCompactFiles.size()); - for(HStoreFile hsf: toCompactFiles) { + out.writeInt(filesToCompact.size()); + for (HStoreFile hsf: filesToCompact) { hsf.write(out); } } finally { @@ -657,7 +1043,8 @@ class HStore implements HConstants { (new DataOutputStream(fs.create(doneFile))).close(); // Move the compaction into place. - processReadyCompaction(); + completeCompaction(); + return true; } finally { // Clean up the parent -- the region dir in the compactions directory. if (this.fs.exists(curCompactStore.getParent())) { @@ -670,24 +1057,6 @@ class HStore implements HConstants { } } - /* - * @return list of files to compact sorted so most recent comes first. - */ - private List getFilesToCompact() { - List filesToCompact = null; - this.lock.obtainWriteLock(); - try { - // Storefiles are keyed by sequence id. The oldest file comes first. - // We need to return out of here a List that has the newest file as - // first. - filesToCompact = new ArrayList(this.storefiles.values()); - Collections.reverse(filesToCompact); - } finally { - this.lock.releaseWriteLock(); - } - return filesToCompact; - } - /* * Compact passed toCompactFiles into compactedOut. * We create a new set of MapFile.Reader objects so we don't screw up @@ -704,9 +1073,9 @@ class HStore implements HConstants { * @param toCompactFiles * @throws IOException */ - void compact(final MapFile.Writer compactedOut, - final List toCompactFiles) - throws IOException { + private void compactHStoreFiles(final MapFile.Writer compactedOut, + final List toCompactFiles) throws IOException { + int size = toCompactFiles.size(); CompactionReader[] rdrs = new CompactionReader[size]; int index = 0; @@ -724,7 +1093,89 @@ class HStore implements HConstants { } } try { - compact(compactedOut, rdrs); + HStoreKey[] keys = new HStoreKey[rdrs.length]; + ImmutableBytesWritable[] vals = new ImmutableBytesWritable[rdrs.length]; + boolean[] done = new boolean[rdrs.length]; + for(int i = 0; i < rdrs.length; i++) { + keys[i] = new HStoreKey(); + vals[i] = new ImmutableBytesWritable(); + done[i] = false; + } + + // Now, advance through the readers in order. This will have the + // effect of a run-time sort of the entire dataset. + int numDone = 0; + for(int i = 0; i < rdrs.length; i++) { + rdrs[i].reset(); + done[i] = ! rdrs[i].next(keys[i], vals[i]); + if(done[i]) { + numDone++; + } + } + + int timesSeen = 0; + Text lastRow = new Text(); + Text lastColumn = new Text(); + // Map of a row deletes keyed by column with a list of timestamps for value + Map> deletes = null; + while (numDone < done.length) { + // Find the reader with the smallest key. If two files have same key + // but different values -- i.e. one is delete and other is non-delete + // value -- we will find the first, the one that was written later and + // therefore the one whose value should make it out to the compacted + // store file. + int smallestKey = -1; + for(int i = 0; i < rdrs.length; i++) { + if(done[i]) { + continue; + } + if(smallestKey < 0) { + smallestKey = i; + } else { + if(keys[i].compareTo(keys[smallestKey]) < 0) { + smallestKey = i; + } + } + } + + // Reflect the current key/val in the output + HStoreKey sk = keys[smallestKey]; + if(lastRow.equals(sk.getRow()) + && lastColumn.equals(sk.getColumn())) { + timesSeen++; + } else { + timesSeen = 1; + // We are on to a new row. Create a new deletes list. + deletes = new HashMap>(); + } + + byte [] value = (vals[smallestKey] == null)? + null: vals[smallestKey].get(); + if (!isDeleted(sk, value, false, deletes) && + timesSeen <= family.getMaxVersions()) { + // Keep old versions until we have maxVersions worth. + // Then just skip them. + if (sk.getRow().getLength() != 0 && sk.getColumn().getLength() != 0) { + // Only write out objects which have a non-zero length key and + // value + compactedOut.append(sk, vals[smallestKey]); + } + } + + // Update last-seen items + lastRow.set(sk.getRow()); + lastColumn.set(sk.getColumn()); + + // Advance the smallest key. If that reader's all finished, then + // mark it as done. + if(!rdrs[smallestKey].next(keys[smallestKey], + vals[smallestKey])) { + done[smallestKey] = true; + rdrs[smallestKey].close(); + rdrs[smallestKey] = null; + numDone++; + } + } } finally { for (int i = 0; i < rdrs.length; i++) { if (rdrs[i] != null) { @@ -789,131 +1240,6 @@ class HStore implements HConstants { this.reader.reset(); } } - - void compact(final MapFile.Writer compactedOut, - final Iterator> iterator, - final MapFile.Reader reader) - throws IOException { - // Make an instance of a CompactionReader that wraps the iterator. - CompactionReader cr = new CompactionReader() { - public boolean next(WritableComparable key, Writable val) - throws IOException { - boolean result = false; - while (iterator.hasNext()) { - Entry e = iterator.next(); - HStoreKey hsk = e.getKey(); - if (familyName.equals(HStoreKey.extractFamily(hsk.getColumn()))) { - ((HStoreKey)key).set(hsk); - ((ImmutableBytesWritable)val).set(e.getValue()); - result = true; - break; - } - } - return result; - } - - @SuppressWarnings("unused") - public void reset() throws IOException { - // noop. - } - - @SuppressWarnings("unused") - public void close() throws IOException { - // noop. - } - }; - - compact(compactedOut, - new CompactionReader [] {cr, new MapFileCompactionReader(reader)}); - } - - void compact(final MapFile.Writer compactedOut, - final CompactionReader [] rdrs) - throws IOException { - HStoreKey[] keys = new HStoreKey[rdrs.length]; - ImmutableBytesWritable[] vals = new ImmutableBytesWritable[rdrs.length]; - boolean[] done = new boolean[rdrs.length]; - for(int i = 0; i < rdrs.length; i++) { - keys[i] = new HStoreKey(); - vals[i] = new ImmutableBytesWritable(); - done[i] = false; - } - - // Now, advance through the readers in order. This will have the - // effect of a run-time sort of the entire dataset. - int numDone = 0; - for(int i = 0; i < rdrs.length; i++) { - rdrs[i].reset(); - done[i] = ! rdrs[i].next(keys[i], vals[i]); - if(done[i]) { - numDone++; - } - } - - int timesSeen = 0; - Text lastRow = new Text(); - Text lastColumn = new Text(); - // Map of a row deletes keyed by column with a list of timestamps for value - Map> deletes = null; - while (numDone < done.length) { - // Find the reader with the smallest key. If two files have same key - // but different values -- i.e. one is delete and other is non-delete - // value -- we will find the first, the one that was written later and - // therefore the one whose value should make it out to the compacted - // store file. - int smallestKey = -1; - for(int i = 0; i < rdrs.length; i++) { - if(done[i]) { - continue; - } - if(smallestKey < 0) { - smallestKey = i; - } else { - if(keys[i].compareTo(keys[smallestKey]) < 0) { - smallestKey = i; - } - } - } - - // Reflect the current key/val in the output - HStoreKey sk = keys[smallestKey]; - if(lastRow.equals(sk.getRow()) - && lastColumn.equals(sk.getColumn())) { - timesSeen++; - } else { - timesSeen = 1; - // We are on to a new row. Create a new deletes list. - deletes = new HashMap>(); - } - - byte [] value = (vals[smallestKey] == null)? - null: vals[smallestKey].get(); - if (!isDeleted(sk, value, null, deletes) && - timesSeen <= family.getMaxVersions()) { - // Keep old versions until we have maxVersions worth. - // Then just skip them. - if (sk.getRow().getLength() != 0 && sk.getColumn().getLength() != 0) { - // Only write out objects which have a non-zero length key and - // value - compactedOut.append(sk, vals[smallestKey]); - } - } - - // Update last-seen items - lastRow.set(sk.getRow()); - lastColumn.set(sk.getColumn()); - - // Advance the smallest key. If that reader's all finished, then - // mark it as done. - if(!rdrs[smallestKey].next(keys[smallestKey], - vals[smallestKey])) { - done[smallestKey] = true; - rdrs[smallestKey].close(); - rdrs[smallestKey] = null; - numDone++; - } - } - } /* * Check if this is cell is deleted. @@ -923,7 +1249,7 @@ class HStore implements HConstants { * deletes map. * @param hsk * @param value - * @param memcache Can be null. + * @param checkMemcache true if the memcache should be consulted * @param deletes Map keyed by column with a value of timestamp. Can be null. * If non-null and passed value is HGlobals.deleteBytes, then we add to this * map. @@ -931,14 +1257,14 @@ class HStore implements HConstants { * passed value is HGlobals.deleteBytes. */ private boolean isDeleted(final HStoreKey hsk, final byte [] value, - final HMemcache memcache, final Map> deletes) { - if (memcache != null && memcache.isDeleted(hsk)) { + final boolean checkMemcache, final Map> deletes) { + if (checkMemcache && memcache.isDeleted(hsk)) { return true; } - List timestamps = (deletes == null)? - null: deletes.get(hsk.getColumn()); + List timestamps = + (deletes == null) ? null: deletes.get(hsk.getColumn()); if (timestamps != null && - timestamps.contains(Long.valueOf(hsk.getTimestamp()))) { + timestamps.contains(Long.valueOf(hsk.getTimestamp()))) { return true; } if (value == null) { @@ -972,97 +1298,117 @@ class HStore implements HConstants { * *

Moving the compacted TreeMap into place means: *

-   * 1) Acquiring the write-lock
-   * 2) Figuring out what MapFiles are going to be replaced
-   * 3) Moving the new compacted MapFile into place
-   * 4) Unloading all the replaced MapFiles.
-   * 5) Deleting all the old MapFile files.
-   * 6) Loading the new TreeMap.
-   * 7) Releasing the write-lock
+   * 1) Wait for active scanners to exit
+   * 2) Acquiring the write-lock
+   * 3) Figuring out what MapFiles are going to be replaced
+   * 4) Moving the new compacted MapFile into place
+   * 5) Unloading all the replaced MapFiles.
+   * 6) Deleting all the old MapFile files.
+   * 7) Loading the new TreeMap.
+   * 8) Releasing the write-lock
+   * 9) Allow new scanners to proceed.
    * 
*/ - void processReadyCompaction() throws IOException { - // 1. Acquiring the write-lock + private void completeCompaction() throws IOException { Path curCompactStore = HStoreFile.getHStoreDir(this.compactionDir, encodedRegionName, familyName); - this.lock.obtainWriteLock(); + + // 1. Wait for active scanners to exit + newScannerLock.writeLock().lock(); // prevent new scanners try { - Path doneFile = new Path(curCompactStore, COMPACTION_DONE); - if (!fs.exists(doneFile)) { - // The last execution didn't finish the compaction, so there's nothing - // we can do. We'll just have to redo it. Abandon it and return. - LOG.warn("Redo failed compaction (missing 'done' file)"); - return; - } - - // 2. Load in the files to be deleted. - Vector toCompactFiles = new Vector(); - Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE); - DataInputStream in = new DataInputStream(fs.open(filesToReplace)); - try { - int numfiles = in.readInt(); - for(int i = 0; i < numfiles; i++) { - HStoreFile hsf = new HStoreFile(conf); - hsf.readFields(in); - toCompactFiles.add(hsf); + synchronized (activeScanners) { + while (activeScanners.get() != 0) { + try { + activeScanners.wait(); + } catch (InterruptedException e) { + // continue + } + } + + // 2. Acquiring the HStore write-lock + this.lock.writeLock().lock(); + } + try { + Path doneFile = new Path(curCompactStore, COMPACTION_DONE); + if (!fs.exists(doneFile)) { + // The last execution didn't finish the compaction, so there's nothing + // we can do. We'll just have to redo it. Abandon it and return. + LOG.warn("Redo failed compaction (missing 'done' file)"); + return; + } + + // 3. Load in the files to be deleted. + Vector toCompactFiles = new Vector(); + Path filesToReplace = new Path(curCompactStore, COMPACTION_TO_REPLACE); + DataInputStream in = new DataInputStream(fs.open(filesToReplace)); + try { + int numfiles = in.readInt(); + for(int i = 0; i < numfiles; i++) { + HStoreFile hsf = new HStoreFile(conf); + hsf.readFields(in); + toCompactFiles.add(hsf); + } + + } finally { + in.close(); + } + + // 4. Moving the new MapFile into place. + HStoreFile compactedFile = new HStoreFile(conf, this.compactionDir, + encodedRegionName, familyName, -1); + // obtainNewHStoreFile does its best to generate a filename that does not + // currently exist. + HStoreFile finalCompactedFile = HStoreFile.obtainNewHStoreFile(conf, dir, + encodedRegionName, familyName, fs); + if(LOG.isDebugEnabled()) { + LOG.debug("moving " + compactedFile.toString() + " in " + + this.compactionDir.toString() + + " to " + finalCompactedFile.toString() + " in " + dir.toString()); + } + if (!compactedFile.rename(this.fs, finalCompactedFile)) { + LOG.error("Failed move of compacted file " + + finalCompactedFile.toString()); + return; + } + + // 5. and 6. Unload all the replaced MapFiles, close and delete. + Vector toDelete = new Vector(toCompactFiles.size()); + for (Map.Entry e: this.storefiles.entrySet()) { + if (!toCompactFiles.contains(e.getValue())) { + continue; + } + Long key = e.getKey(); + MapFile.Reader reader = this.readers.remove(key); + if (reader != null) { + reader.close(); + } + toDelete.add(key); + } + + try { + for (Long key: toDelete) { + HStoreFile hsf = this.storefiles.remove(key); + hsf.delete(); + } + + // 7. Loading the new TreeMap. + Long orderVal = Long.valueOf(finalCompactedFile.loadInfo(fs)); + this.readers.put(orderVal, + finalCompactedFile.getReader(this.fs, this.bloomFilter)); + this.storefiles.put(orderVal, finalCompactedFile); + } catch (IOException e) { + LOG.error("Failed replacing compacted files. Compacted file is " + + finalCompactedFile.toString() + ". Files replaced are " + + toCompactFiles.toString() + + " some of which may have been already removed", e); } - } finally { - in.close(); - } - - // 3. Moving the new MapFile into place. - HStoreFile compactedFile = new HStoreFile(conf, this.compactionDir, - encodedRegionName, familyName, -1); - // obtainNewHStoreFile does its best to generate a filename that does not - // currently exist. - HStoreFile finalCompactedFile = HStoreFile.obtainNewHStoreFile(conf, dir, - encodedRegionName, familyName, fs); - if(LOG.isDebugEnabled()) { - LOG.debug("moving " + compactedFile.toString() + " in " + - this.compactionDir.toString() + - " to " + finalCompactedFile.toString() + " in " + dir.toString()); - } - if (!compactedFile.rename(this.fs, finalCompactedFile)) { - LOG.error("Failed move of compacted file " + - finalCompactedFile.toString()); - return; - } - - // 4. and 5. Unload all the replaced MapFiles, close and delete. - Vector toDelete = new Vector(toCompactFiles.size()); - for (Map.Entry e: this.storefiles.entrySet()) { - if (!toCompactFiles.contains(e.getValue())) { - continue; - } - Long key = e.getKey(); - MapFile.Reader reader = this.readers.remove(key); - if (reader != null) { - reader.close(); - } - toDelete.add(key); - } - - try { - for (Long key: toDelete) { - HStoreFile hsf = this.storefiles.remove(key); - hsf.delete(); - } - - // 6. Loading the new TreeMap. - Long orderVal = Long.valueOf(finalCompactedFile.loadInfo(fs)); - this.readers.put(orderVal, - finalCompactedFile.getReader(this.fs, this.bloomFilter)); - this.storefiles.put(orderVal, finalCompactedFile); - } catch (IOException e) { - LOG.error("Failed replacing compacted files. Compacted file is " + - finalCompactedFile.toString() + ". Files replaced are " + - toCompactFiles.toString() + - " some of which may have been already removed", e); + // 8. Releasing the write-lock + this.lock.writeLock().unlock(); } } finally { - // 7. Releasing the write-lock - this.lock.releaseWriteLock(); + // 9. Allow new scanners to proceed. + newScannerLock.writeLock().unlock(); } } @@ -1078,8 +1424,10 @@ class HStore implements HConstants { * The returned object should map column names to byte arrays (byte[]). */ void getFull(HStoreKey key, TreeMap results) - throws IOException { - this.lock.obtainReadLock(); + throws IOException { + + this.lock.readLock().lock(); + memcache.getFull(key, results); try { MapFile.Reader[] maparray = getReaders(); for (int i = maparray.length - 1; i >= 0; i--) { @@ -1109,7 +1457,7 @@ class HStore implements HConstants { } } finally { - this.lock.releaseReadLock(); + this.lock.readLock().unlock(); } } @@ -1125,29 +1473,33 @@ class HStore implements HConstants { * If 'numVersions' is negative, the method returns all available versions. * @param key * @param numVersions Number of versions to fetch. Must be > 0. - * @param memcache Checked for deletions * @return values for the specified versions * @throws IOException */ - byte [][] get(HStoreKey key, int numVersions, final HMemcache memcache) - throws IOException { + byte [][] get(HStoreKey key, int numVersions) throws IOException { if (numVersions <= 0) { throw new IllegalArgumentException("Number of versions must be > 0"); } - List results = new ArrayList(); - // Keep a list of deleted cell keys. We need this because as we go through - // the store files, the cell with the delete marker may be in one file and - // the old non-delete cell value in a later store file. If we don't keep - // around the fact that the cell was deleted in a newer record, we end up - // returning the old value if user is asking for more than one version. - // This List of deletes should not large since we are only keeping rows - // and columns that match those set on the scanner and which have delete - // values. If memory usage becomes an issue, could redo as bloom filter. - Map> deletes = new HashMap>(); - // This code below is very close to the body of the getKeys method. - this.lock.obtainReadLock(); + this.lock.readLock().lock(); try { + // Check the memcache + List results = this.memcache.get(key, numVersions); + // If we got sufficient versions from memcache, return. + if (results.size() == numVersions) { + return ImmutableBytesWritable.toArray(results); + } + + // Keep a list of deleted cell keys. We need this because as we go through + // the store files, the cell with the delete marker may be in one file and + // the old non-delete cell value in a later store file. If we don't keep + // around the fact that the cell was deleted in a newer record, we end up + // returning the old value if user is asking for more than one version. + // This List of deletes should not large since we are only keeping rows + // and columns that match those set on the scanner and which have delete + // values. If memory usage becomes an issue, could redo as bloom filter. + Map> deletes = new HashMap>(); + // This code below is very close to the body of the getKeys method. MapFile.Reader[] maparray = getReaders(); for(int i = maparray.length - 1; i >= 0; i--) { MapFile.Reader map = maparray[i]; @@ -1165,7 +1517,7 @@ class HStore implements HConstants { if (!readkey.matchesRowCol(key)) { continue; } - if (!isDeleted(readkey, readval.get(), memcache, deletes)) { + if (!isDeleted(readkey, readval.get(), true, deletes)) { results.add(readval.get()); // Perhaps only one version is wanted. I could let this // test happen later in the for loop test but it would cost @@ -1179,7 +1531,7 @@ class HStore implements HConstants { readkey.matchesRowCol(key) && !hasEnoughVersions(numVersions, results); readval = new ImmutableBytesWritable()) { - if (!isDeleted(readkey, readval.get(), memcache, deletes)) { + if (!isDeleted(readkey, readval.get(), true, deletes)) { results.add(readval.get()); } } @@ -1191,7 +1543,7 @@ class HStore implements HConstants { return results.size() == 0 ? null : ImmutableBytesWritable.toArray(results); } finally { - this.lock.releaseReadLock(); + this.lock.readLock().unlock(); } } @@ -1215,15 +1567,16 @@ class HStore implements HConstants { * matching keys found in store files appended. * @throws IOException */ - List getKeys(final HStoreKey origin, List allKeys, - final int versions) throws IOException { + List getKeys(final HStoreKey origin, final int versions) + throws IOException { - List keys = allKeys; - if (keys == null) { - keys = new ArrayList(); + List keys = this.memcache.getKeys(origin, versions); + if (versions != ALL_VERSIONS && keys.size() >= versions) { + return keys; } + // This code below is very close to the body of the get method. - this.lock.obtainReadLock(); + this.lock.readLock().lock(); try { MapFile.Reader[] maparray = getReaders(); for(int i = maparray.length - 1; i >= 0; i--) { @@ -1242,7 +1595,7 @@ class HStore implements HConstants { if (!readkey.matchesRowCol(origin)) { continue; } - if (!isDeleted(readkey, readval.get(), null, null) && + if (!isDeleted(readkey, readval.get(), false, null) && !keys.contains(readkey)) { keys.add(new HStoreKey(readkey)); } @@ -1250,7 +1603,7 @@ class HStore implements HConstants { map.next(readkey, readval) && readkey.matchesRowCol(origin); readval = new ImmutableBytesWritable()) { - if (!isDeleted(readkey, readval.get(), null, null) && + if (!isDeleted(readkey, readval.get(), false, null) && !keys.contains(readkey)) { keys.add(new HStoreKey(readkey)); if (versions != ALL_VERSIONS && keys.size() >= versions) { @@ -1262,7 +1615,7 @@ class HStore implements HConstants { } return keys; } finally { - this.lock.releaseReadLock(); + this.lock.readLock().unlock(); } } @@ -1315,7 +1668,7 @@ class HStore implements HConstants { return new HStoreSize(0, 0, splitable); } - this.lock.obtainReadLock(); + this.lock.readLock().lock(); try { Long mapIndex = Long.valueOf(0L); // Iterate through all the MapFiles @@ -1340,84 +1693,52 @@ class HStore implements HConstants { } catch(IOException e) { LOG.warn("Failed getting store size", e); } finally { - this.lock.releaseReadLock(); + this.lock.readLock().unlock(); } return new HStoreSize(aggregateSize, maxSize, splitable); } - /** - * @return Returns the number of map files currently in use - */ - int countOfStoreFiles() { - this.lock.obtainReadLock(); - try { - return storefiles.size(); - - } finally { - this.lock.releaseReadLock(); - } - } - - boolean hasReferences() { - boolean result = false; - this.lock.obtainReadLock(); - try { - for (HStoreFile hsf: this.storefiles.values()) { - if (hsf.isReference()) { - break; - } - } - - } finally { - this.lock.releaseReadLock(); - } - return result; - } - ////////////////////////////////////////////////////////////////////////////// // File administration ////////////////////////////////////////////////////////////////////////////// - /** Generate a random unique filename suffix */ - String obtainFileLabel(Path prefix) throws IOException { - String testsuffix = String.valueOf(rand.nextInt(Integer.MAX_VALUE)); - Path testpath = new Path(prefix.toString() + testsuffix); - while(fs.exists(testpath)) { - testsuffix = String.valueOf(rand.nextInt(Integer.MAX_VALUE)); - testpath = new Path(prefix.toString() + testsuffix); - } - return testsuffix; - } - /** - * Return a set of MapFile.Readers, one for each HStore file. - * These should be closed after the user is done with them. + * Return a scanner for both the memcache and the HStore files */ HInternalScannerInterface getScanner(long timestamp, Text targetCols[], - Text firstRow) throws IOException { - - return new HStoreScanner(timestamp, targetCols, firstRow); + Text firstRow, RowFilterInterface filter) throws IOException { + + newScannerLock.readLock().lock(); // ability to create a new + // scanner during a compaction + try { + lock.readLock().lock(); // lock HStore + try { + return new HStoreScanner(targetCols, firstRow, timestamp, filter); + + } finally { + lock.readLock().unlock(); + } + } finally { + newScannerLock.readLock().unlock(); + } } - + /** {@inheritDoc} */ @Override public String toString() { return this.storeName; } - ////////////////////////////////////////////////////////////////////////////// - // This class implements the HScannerInterface. - // It lets the caller scan the contents of this HStore. - ////////////////////////////////////////////////////////////////////////////// - - class HStoreScanner extends HAbstractScanner { + /** + * A scanner that iterates through the HStore files + */ + private class StoreFileScanner extends HAbstractScanner { @SuppressWarnings("hiding") private MapFile.Reader[] readers; - HStoreScanner(long timestamp, Text[] targetCols, Text firstRow) + StoreFileScanner(long timestamp, Text[] targetCols, Text firstRow) throws IOException { super(timestamp, targetCols); - lock.obtainReadLock(); try { this.readers = new MapFile.Reader[storefiles.size()]; @@ -1430,7 +1751,7 @@ class HStore implements HConstants { this.keys = new HStoreKey[readers.length]; this.vals = new byte[readers.length][]; - + // Advance the readers to the first pos. for(i = 0; i < readers.length; i++) { keys[i] = new HStoreKey(); @@ -1539,10 +1860,281 @@ class HStore implements HConstants { } } finally { - lock.releaseReadLock(); scannerClosed = true; } } } } + + /** + * Scanner scans both the memcache and the HStore + */ + private class HStoreScanner implements HInternalScannerInterface { + private HInternalScannerInterface[] scanners; + private TreeMap[] resultSets; + private HStoreKey[] keys; + private boolean wildcardMatch = false; + private boolean multipleMatchers = false; + private RowFilterInterface dataFilter; + + /** Create an Scanner with a handle on the memcache and HStore files. */ + @SuppressWarnings("unchecked") + HStoreScanner(Text[] targetCols, Text firstRow, long timestamp, + RowFilterInterface filter) throws IOException { + + this.dataFilter = filter; + if (null != dataFilter) { + dataFilter.reset(); + } + this.scanners = new HInternalScannerInterface[2]; + this.resultSets = new TreeMap[scanners.length]; + this.keys = new HStoreKey[scanners.length]; + + try { + scanners[0] = memcache.getScanner(timestamp, targetCols, firstRow); + scanners[1] = new StoreFileScanner(timestamp, targetCols, firstRow); + + for (int i = 0; i < scanners.length; i++) { + if (scanners[i].isWildcardScanner()) { + this.wildcardMatch = true; + } + if (scanners[i].isMultipleMatchScanner()) { + this.multipleMatchers = true; + } + } + + } catch(IOException e) { + for (int i = 0; i < this.scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + throw e; + } + + // Advance to the first key in each scanner. + // All results will match the required column-set and scanTime. + + for (int i = 0; i < scanners.length; i++) { + keys[i] = new HStoreKey(); + resultSets[i] = new TreeMap(); + if(scanners[i] != null && !scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + // As we have now successfully completed initialization, increment the + // activeScanner count. + activeScanners.incrementAndGet(); + } + + /** @return true if the scanner is a wild card scanner */ + public boolean isWildcardScanner() { + return wildcardMatch; + } + + /** @return true if the scanner is a multiple match scanner */ + public boolean isMultipleMatchScanner() { + return multipleMatchers; + } + + /** {@inheritDoc} */ + public boolean next(HStoreKey key, SortedMap results) + throws IOException { + + // Filtered flag is set by filters. If a cell has been 'filtered out' + // -- i.e. it is not to be returned to the caller -- the flag is 'true'. + boolean filtered = true; + boolean moreToFollow = true; + while (filtered && moreToFollow) { + // Find the lowest-possible key. + Text chosenRow = null; + long chosenTimestamp = -1; + for (int i = 0; i < this.keys.length; i++) { + if (scanners[i] != null && + (chosenRow == null || + (keys[i].getRow().compareTo(chosenRow) < 0) || + ((keys[i].getRow().compareTo(chosenRow) == 0) && + (keys[i].getTimestamp() > chosenTimestamp)))) { + chosenRow = new Text(keys[i].getRow()); + chosenTimestamp = keys[i].getTimestamp(); + } + } + + // Filter whole row by row key? + filtered = dataFilter != null? dataFilter.filter(chosenRow) : false; + + // Store the key and results for each sub-scanner. Merge them as + // appropriate. + if (chosenTimestamp >= 0 && !filtered) { + // Here we are setting the passed in key with current row+timestamp + key.setRow(chosenRow); + key.setVersion(chosenTimestamp); + key.setColumn(HConstants.EMPTY_TEXT); + // Keep list of deleted cell keys within this row. We need this + // because as we go through scanners, the delete record may be in an + // early scanner and then the same record with a non-delete, non-null + // value in a later. Without history of what we've seen, we'll return + // deleted values. This List should not ever grow too large since we + // are only keeping rows and columns that match those set on the + // scanner and which have delete values. If memory usage becomes a + // problem, could redo as bloom filter. + List deletes = new ArrayList(); + for (int i = 0; i < scanners.length && !filtered; i++) { + while ((scanners[i] != null + && !filtered + && moreToFollow) + && (keys[i].getRow().compareTo(chosenRow) == 0)) { + // If we are doing a wild card match or there are multiple + // matchers per column, we need to scan all the older versions of + // this row to pick up the rest of the family members + if (!wildcardMatch + && !multipleMatchers + && (keys[i].getTimestamp() != chosenTimestamp)) { + break; + } + + // Filter out null criteria columns that are not null + if (dataFilter != null) { + filtered = dataFilter.filterNotNull(resultSets[i]); + } + + // NOTE: We used to do results.putAll(resultSets[i]); + // but this had the effect of overwriting newer + // values with older ones. So now we only insert + // a result if the map does not contain the key. + HStoreKey hsk = new HStoreKey(key.getRow(), EMPTY_TEXT, + key.getTimestamp()); + for (Map.Entry e : resultSets[i].entrySet()) { + hsk.setColumn(e.getKey()); + if (HLogEdit.isDeleted(e.getValue())) { + if (!deletes.contains(hsk)) { + // Key changes as we cycle the for loop so add a copy to + // the set of deletes. + deletes.add(new HStoreKey(hsk)); + } + } else if (!deletes.contains(hsk) && + !filtered && + moreToFollow && + !results.containsKey(e.getKey())) { + if (dataFilter != null) { + // Filter whole row by column data? + filtered = + dataFilter.filter(chosenRow, e.getKey(), e.getValue()); + if (filtered) { + results.clear(); + break; + } + } + results.put(e.getKey(), e.getValue()); + } + } + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + } + + for (int i = 0; i < scanners.length; i++) { + // If the current scanner is non-null AND has a lower-or-equal + // row label, then its timestamp is bad. We need to advance it. + while ((scanners[i] != null) && + (keys[i].getRow().compareTo(chosenRow) <= 0)) { + resultSets[i].clear(); + if (!scanners[i].next(keys[i], resultSets[i])) { + closeScanner(i); + } + } + } + + moreToFollow = chosenTimestamp >= 0; + + if (dataFilter != null) { + if (moreToFollow) { + dataFilter.rowProcessed(filtered, chosenRow); + } + if (dataFilter.filterAllRemaining()) { + moreToFollow = false; + LOG.debug("page limit"); + } + } + if (LOG.isDebugEnabled()) { + if (this.dataFilter != null) { + LOG.debug("ROWKEY = " + chosenRow + ", FILTERED = " + filtered); + } + } + + if (results.size() <= 0 && !filtered) { + // There were no results found for this row. Marked it as + // 'filtered'-out otherwise we will not move on to the next row. + filtered = true; + } + } + + // If we got no results, then there is no more to follow. + if (results == null || results.size() <= 0) { + moreToFollow = false; + } + + // Make sure scanners closed if no more results + if (!moreToFollow) { + for (int i = 0; i < scanners.length; i++) { + if (null != scanners[i]) { + closeScanner(i); + } + } + } + + return moreToFollow; + } + + + /** Shut down a single scanner */ + void closeScanner(int i) { + try { + try { + scanners[i].close(); + } catch (IOException e) { + LOG.warn("Failed closeing scanner " + i, e); + } + } finally { + scanners[i] = null; + keys[i] = null; + resultSets[i] = null; + } + } + + /** {@inheritDoc} */ + public void close() { + try { + for(int i = 0; i < scanners.length; i++) { + if(scanners[i] != null) { + closeScanner(i); + } + } + } finally { + synchronized (activeScanners) { + int numberOfScanners = activeScanners.decrementAndGet(); + if (numberOfScanners < 0) { + LOG.error("number of active scanners less than zero: " + + numberOfScanners + " resetting to zero"); + activeScanners.set(0); + numberOfScanners = 0; + } + if (numberOfScanners == 0) { + activeScanners.notifyAll(); + } + } + + } + } + + /** {@inheritDoc} */ + public Iterator>> iterator() { + throw new UnsupportedOperationException("Unimplemented serverside. " + + "next(HStoreKey, StortedMap(...) is more efficient"); + } + } + } diff --git a/src/java/org/apache/hadoop/hbase/HTable.java b/src/java/org/apache/hadoop/hbase/HTable.java index e8edd7dec59..f254424a85a 100644 --- a/src/java/org/apache/hadoop/hbase/HTable.java +++ b/src/java/org/apache/hadoop/hbase/HTable.java @@ -937,6 +937,7 @@ public class HTable implements HConstants { this.closed = true; } + /** {@inheritDoc} */ public Iterator>> iterator() { return new Iterator>>() { HStoreKey key = null; diff --git a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java index 8e0a934d737..4dfe46e0f23 100644 --- a/src/java/org/apache/hadoop/hbase/io/BatchOperation.java +++ b/src/java/org/apache/hadoop/hbase/io/BatchOperation.java @@ -34,7 +34,11 @@ public class BatchOperation implements Writable { * Operation types. * @see org.apache.hadoop.io.SequenceFile.Writer */ - public static enum Operation {PUT, DELETE} + public static enum Operation { + /** update a field */ + PUT, + /** delete a field */ + DELETE} private Operation op; private Text column; @@ -65,7 +69,8 @@ public class BatchOperation implements Writable { /** * Creates a put operation - * + * + * @param operation the operation (put or get) * @param column column name * @param value column value */ diff --git a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java index 09a0186ab06..fd9c0ef9635 100644 --- a/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java +++ b/src/test/org/apache/hadoop/hbase/AbstractMergeTestBase.java @@ -102,9 +102,7 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase { meta.getLog().closeAndDelete(); } catch (Exception e) { - if(dfsCluster != null) { - dfsCluster.shutdown(); - } + StaticTestEnvironment.shutdownDfs(dfsCluster); throw e; } } @@ -115,16 +113,7 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase { @Override public void tearDown() throws Exception { super.tearDown(); - if (dfsCluster != null) { - dfsCluster.shutdown(); - } - if (this.fs != null) { - try { - this.fs.close(); - } catch (IOException e) { - LOG.info("During tear down got a " + e.getMessage()); - } - } + StaticTestEnvironment.shutdownDfs(dfsCluster); } private HRegion createAregion(Text startKey, Text endKey, int firstRow, @@ -134,20 +123,21 @@ public abstract class AbstractMergeTestBase extends HBaseTestCase { System.out.println("created region " + region.getRegionName()); + HRegionIncommon r = new HRegionIncommon(region); for(int i = firstRow; i < firstRow + nrows; i++) { - long lockid = region.startUpdate(new Text("row_" + long lockid = r.startUpdate(new Text("row_" + String.format("%1$05d", i))); - region.put(lockid, COLUMN_NAME, value.get()); - region.commit(lockid, System.currentTimeMillis()); + r.put(lockid, COLUMN_NAME, value.get()); + r.commit(lockid, System.currentTimeMillis()); if(i % 10000 == 0) { System.out.println("Flushing write #" + i); - region.flushcache(false); + r.flushcache(); } } System.out.println("Rolling log..."); region.log.rollWriter(); - region.compactStores(); + region.compactIfNeeded(); region.close(); region.getLog().closeAndDelete(); region.getRegionInfo().setOffline(true); diff --git a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java index f03337b8381..9d96d5235ca 100644 --- a/src/test/org/apache/hadoop/hbase/HBaseTestCase.java +++ b/src/test/org/apache/hadoop/hbase/HBaseTestCase.java @@ -21,6 +21,8 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.io.UnsupportedEncodingException; +import java.util.Map; +import java.util.Random; import junit.framework.TestCase; @@ -28,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HColumnDescriptor.CompressionType; +import org.apache.hadoop.hbase.io.BatchUpdate; import org.apache.hadoop.io.Text; /** @@ -219,19 +222,36 @@ public abstract class HBaseTestCase extends TestCase { && endKey.compareTo(t) <= 0) { break EXIT; } - long lockid = updater.startBatchUpdate(t); try { - updater.put(lockid, new Text(column), bytes); - if (ts == -1) { - updater.commit(lockid); - } else { - updater.commit(lockid, ts); - } - lockid = -1; - } finally { - if (lockid != -1) { - updater.abort(lockid); + long lockid = updater.startBatchUpdate(t); + try { + updater.put(lockid, new Text(column), bytes); + if (ts == -1) { + updater.commit(lockid); + } else { + updater.commit(lockid, ts); + } + lockid = -1; + } catch (RuntimeException ex) { + ex.printStackTrace(); + throw ex; + + } catch (IOException ex) { + ex.printStackTrace(); + throw ex; + + } finally { + if (lockid != -1) { + updater.abort(lockid); + } } + } catch (RuntimeException ex) { + ex.printStackTrace(); + throw ex; + + } catch (IOException ex) { + ex.printStackTrace(); + throw ex; } } // Set start character back to FIRST_CHAR after we've done first loop. @@ -275,32 +295,56 @@ public abstract class HBaseTestCase extends TestCase { /** * A class that makes a {@link Incommon} out of a {@link HRegion} */ - public static class HRegionIncommon implements Incommon { + public static class HRegionIncommon implements Incommon, FlushCache { final HRegion region; + private final Random rand = new Random(); + private BatchUpdate batch; + + private void checkBatch() { + if (batch == null) { + throw new IllegalStateException("No update in progress"); + } + } + public HRegionIncommon(final HRegion HRegion) { - super(); this.region = HRegion; + this.batch = null; } public void abort(long lockid) throws IOException { - this.region.abort(lockid); + this.batch = null; } public void commit(long lockid) throws IOException { - this.region.commit(lockid); + commit(lockid, HConstants.LATEST_TIMESTAMP); } public void commit(long lockid, final long ts) throws IOException { - this.region.commit(lockid, ts); + checkBatch(); + try { + this.region.batchUpdate(ts, batch); + } finally { + this.batch = null; + } } public void put(long lockid, Text column, byte[] val) throws IOException { - this.region.put(lockid, column, val); + checkBatch(); + this.batch.put(lockid, column, val); } public void delete(long lockid, Text column) throws IOException { - this.region.delete(lockid, column); + checkBatch(); + this.batch.delete(lockid, column); } public void deleteAll(Text row, Text column, long ts) throws IOException { this.region.deleteAll(row, column, ts); } public long startBatchUpdate(Text row) throws IOException { - return this.region.startUpdate(row); + return startUpdate(row); + } + public long startUpdate(Text row) throws IOException { + if (this.batch != null) { + throw new IllegalStateException("Update already in progress"); + } + long lockid = Math.abs(rand.nextLong()); + this.batch = new BatchUpdate(lockid); + return batch.startUpdate(row); } public HScannerInterface getScanner(Text [] columns, Text firstRow, long ts) @@ -317,6 +361,12 @@ public abstract class HBaseTestCase extends TestCase { throws IOException { return this.region.get(row, column, ts, versions); } + public Map getFull(Text row) throws IOException { + return region.getFull(row); + } + public void flushcache() throws IOException { + this.region.internalFlushcache(this.region.snapshotMemcaches()); + } } /** diff --git a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java index 61f7f321cb7..8a12ec0c29c 100644 --- a/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java +++ b/src/test/org/apache/hadoop/hbase/MiniHBaseCluster.java @@ -117,9 +117,14 @@ public class MiniHBaseCluster implements HConstants { this.deleteOnExit = deleteOnExit; this.shutdownDFS = false; if (miniHdfsFilesystem) { - this.cluster = new MiniDFSCluster(this.conf, 2, format, (String[])null); - this.fs = cluster.getFileSystem(); - this.shutdownDFS = true; + try { + this.cluster = new MiniDFSCluster(this.conf, 2, format, (String[])null); + this.fs = cluster.getFileSystem(); + this.shutdownDFS = true; + } catch (IOException e) { + StaticTestEnvironment.shutdownDfs(cluster); + throw e; + } } else { this.cluster = null; this.fs = FileSystem.get(conf); @@ -224,26 +229,13 @@ public class MiniHBaseCluster implements HConstants { */ public void shutdown() { this.hbaseCluster.shutdown(); - try { - if (shutdownDFS && cluster != null) { - FileSystem fs = cluster.getFileSystem(); - if (fs != null) { - LOG.info("Shutting down FileSystem"); - fs.close(); - } - if (this.cluster != null) { - LOG.info("Shutting down Mini DFS "); - cluster.shutdown(); - } - } - } catch (IOException e) { - LOG.error("shutdown", e); - } finally { - // Delete all DFS files - if(deleteOnExit) { - deleteFile(new File(System.getProperty( - StaticTestEnvironment.TEST_DIRECTORY_KEY), "dfs")); - } + if (shutdownDFS) { + StaticTestEnvironment.shutdownDfs(cluster); + } + // Delete all DFS files + if(deleteOnExit) { + deleteFile(new File(System.getProperty( + StaticTestEnvironment.TEST_DIRECTORY_KEY), "dfs")); } } @@ -265,7 +257,7 @@ public class MiniHBaseCluster implements HConstants { for (LocalHBaseCluster.RegionServerThread t: this.hbaseCluster.getRegionServers()) { for(HRegion r: t.getRegionServer().onlineRegions.values() ) { - r.flushcache(false); + r.internalFlushcache(r.snapshotMemcaches()); } } } diff --git a/src/test/org/apache/hadoop/hbase/MultiRegionTable.java b/src/test/org/apache/hadoop/hbase/MultiRegionTable.java index 6e6f2142174..b2a0d1a043d 100644 --- a/src/test/org/apache/hadoop/hbase/MultiRegionTable.java +++ b/src/test/org/apache/hadoop/hbase/MultiRegionTable.java @@ -53,11 +53,9 @@ public class MultiRegionTable extends HBaseTestCase { @SuppressWarnings("null") public static void makeMultiRegionTable(HBaseConfiguration conf, MiniHBaseCluster cluster, FileSystem localFs, String tableName, - String columnName) - throws IOException { + String columnName) throws IOException { final int retries = 10; - final long waitTime = - conf.getLong("hbase.master.meta.thread.rescanfrequency", 10L * 1000L); + final long waitTime = 20L * 1000L; // This size should make it so we always split using the addContent // below. After adding all data, the first region is 1.3M. Should @@ -106,7 +104,7 @@ public class MultiRegionTable extends HBaseTestCase { } // Flush will provoke a split next time the split-checker thread runs. - r.flushcache(false); + r.internalFlushcache(r.snapshotMemcaches()); // Now, wait until split makes it into the meta table. int oldCount = count; @@ -156,15 +154,19 @@ public class MultiRegionTable extends HBaseTestCase { // Wait till the parent only has reference to remaining split, one that // still has references. - while (getSplitParentInfo(meta, parent).size() == 3) { - try { - Thread.sleep(waitTime); - } catch (InterruptedException e) { - // continue + while (true) { + data = getSplitParentInfo(meta, parent); + if (data == null || data.size() == 3) { + try { + Thread.sleep(waitTime); + } catch (InterruptedException e) { + // continue + } + continue; } + break; } - LOG.info("Parent split returned " + - getSplitParentInfo(meta, parent).keySet().toString()); + LOG.info("Parent split returned " + data.keySet().toString()); // Call second split. diff --git a/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java b/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java index b418d38c6e6..dbfb078db77 100644 --- a/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java +++ b/src/test/org/apache/hadoop/hbase/StaticTestEnvironment.java @@ -20,8 +20,11 @@ package org.apache.hadoop.hbase; import java.io.File; +import java.io.IOException; import java.util.Enumeration; +import org.apache.hadoop.dfs.MiniDFSCluster; +import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.Appender; import org.apache.log4j.ConsoleAppender; import org.apache.log4j.Layout; @@ -33,6 +36,9 @@ import org.apache.log4j.PatternLayout; * Initializes test environment */ public class StaticTestEnvironment { + private static final Logger LOG = + Logger.getLogger(StaticTestEnvironment.class.getPackage().getName()); + private StaticTestEnvironment() {} // Not instantiable /** configuration parameter name for test directory */ @@ -105,7 +111,28 @@ public class StaticTestEnvironment { consoleLayout.setConversionPattern("%d %-5p [%t] %l: %m%n"); } } - Logger.getLogger( - HBaseTestCase.class.getPackage().getName()).setLevel(logLevel); + LOG.setLevel(logLevel); + } + + /** + * Common method to close down a MiniDFSCluster and the associated file system + * + * @param cluster + */ + public static void shutdownDfs(MiniDFSCluster cluster) { + if (cluster != null) { + try { + FileSystem fs = cluster.getFileSystem(); + if (fs != null) { + LOG.info("Shutting down FileSystem"); + fs.close(); + } + } catch (IOException e) { + LOG.error("error closing file system", e); + } + + LOG.info("Shutting down Mini DFS "); + cluster.shutdown(); + } } } diff --git a/src/test/org/apache/hadoop/hbase/TestCompaction.java b/src/test/org/apache/hadoop/hbase/TestCompaction.java index 234b7e9ad4f..66391174ae2 100644 --- a/src/test/org/apache/hadoop/hbase/TestCompaction.java +++ b/src/test/org/apache/hadoop/hbase/TestCompaction.java @@ -43,7 +43,11 @@ public class TestCompaction extends HBaseTestCase { /** constructor */ public TestCompaction() { + super(); STARTROW = new Text(START_KEY); + + // Set cache flush size to 1MB + conf.setInt("hbase.hregion.memcache.flush.size", 1024*1024); } /** {@inheritDoc} */ @@ -71,11 +75,10 @@ public class TestCompaction extends HBaseTestCase { */ public void testCompaction() throws Exception { createStoreFile(r); - assertFalse(r.needsCompaction()); + assertFalse(r.compactIfNeeded()); for (int i = 0; i < COMPACTION_THRESHOLD; i++) { createStoreFile(r); } - assertTrue(r.needsCompaction()); // Add more content. Now there are about 5 versions of each column. // Default is that there only 3 (MAXVERSIONS) versions allowed per column. // Assert > 3 and then after compaction, assert that only 3 versions @@ -91,7 +94,7 @@ public class TestCompaction extends HBaseTestCase { @Override public void run() { try { - region.flushcache(false); + region.flushcache(); } catch (IOException e) { e.printStackTrace(); } @@ -101,7 +104,7 @@ public class TestCompaction extends HBaseTestCase { @Override public void run() { try { - assertTrue(region.compactStores()); + assertTrue(region.compactIfNeeded()); } catch (IOException e) { e.printStackTrace(); } @@ -140,16 +143,15 @@ public class TestCompaction extends HBaseTestCase { // verify that it is removed as we compact. // Assert all delted. assertNull(this.r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/)); - this.r.flushcache(false); + this.r.flushcache(); assertNull(this.r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/)); // Add a bit of data and flush it so we for sure have the compaction limit // for store files. Usually by this time we will have but if compaction // included the flush that ran 'concurrently', there may be just the // compacted store and the flush above when we added deletes. Add more // content to be certain. - createBunchOfSmallStoreFiles(this.r); - assertTrue(this.r.needsCompaction()); - this.r.compactStores(); + createSmallerStoreFile(this.r); + assertTrue(this.r.compactIfNeeded()); // Assert that the first row is still deleted. bytes = this.r.get(STARTROW, COLUMN_FAMILY_TEXT, 100 /*Too many*/); assertNull(bytes); @@ -167,21 +169,14 @@ public class TestCompaction extends HBaseTestCase { private void createStoreFile(final HRegion region) throws IOException { HRegionIncommon loader = new HRegionIncommon(region); - for (int i = 0; i < 1; i++) { - addContent(loader, COLUMN_FAMILY); - } - region.flushcache(false); + addContent(loader, COLUMN_FAMILY); + loader.flushcache(); } - private void createBunchOfSmallStoreFiles(final HRegion region) - throws IOException { - final String xyz = new String("xyz"); - byte [] bytes = xyz.getBytes(); - for (int i = 0; i < 1; i++) { - long lid = region.startUpdate(new Text(xyz)); - region.put(lid, COLUMN_FAMILY_TEXT, bytes); - region.commit(lid); - region.flushcache(false); - } + private void createSmallerStoreFile(final HRegion region) throws IOException { + HRegionIncommon loader = new HRegionIncommon(region); + addContent(loader, COLUMN_FAMILY, + ("bbb" + PUNCTUATION).getBytes(), null); + loader.flushcache(); } } diff --git a/src/test/org/apache/hadoop/hbase/TestGet.java b/src/test/org/apache/hadoop/hbase/TestGet.java index a7ff271b178..411158c3443 100644 --- a/src/test/org/apache/hadoop/hbase/TestGet.java +++ b/src/test/org/apache/hadoop/hbase/TestGet.java @@ -23,7 +23,7 @@ import java.io.ByteArrayOutputStream; import java.io.DataOutputStream; import java.io.IOException; import java.util.Iterator; -import java.util.TreeMap; +import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -44,7 +44,7 @@ public class TestGet extends HBaseTestCase { private static final String SERVER_ADDRESS = "foo.bar.com:1234"; - private void verifyGet(final HRegion r, final String expectedServer) + private void verifyGet(final HRegionIncommon r, final String expectedServer) throws IOException { // This should return a value because there is only one family member byte [] value = r.get(ROW_KEY, CONTENTS); @@ -55,7 +55,7 @@ public class TestGet extends HBaseTestCase { assertNull(value); // Find out what getFull returns - TreeMap values = r.getFull(ROW_KEY); + Map values = r.getFull(ROW_KEY); // assertEquals(4, values.keySet().size()); for(Iterator i = values.keySet().iterator(); i.hasNext(); ) { @@ -95,7 +95,8 @@ public class TestGet extends HBaseTestCase { HLog log = new HLog(fs, new Path(regionDir, "log"), conf); - HRegion r = new HRegion(dir, log, fs, conf, info, null); + HRegion region = new HRegion(dir, log, fs, conf, info, null); + HRegionIncommon r = new HRegionIncommon(region); // Write information to the table @@ -132,9 +133,10 @@ public class TestGet extends HBaseTestCase { // Close and re-open region, forcing updates to disk - r.close(); + region.close(); log.rollWriter(); - r = new HRegion(dir, log, fs, conf, info, null); + region = new HRegion(dir, log, fs, conf, info, null); + r = new HRegionIncommon(region); // Read it back @@ -160,9 +162,10 @@ public class TestGet extends HBaseTestCase { // Close region and re-open it - r.close(); + region.close(); log.rollWriter(); - r = new HRegion(dir, log, fs, conf, info, null); + region = new HRegion(dir, log, fs, conf, info, null); + r = new HRegionIncommon(region); // Read it back @@ -170,13 +173,11 @@ public class TestGet extends HBaseTestCase { // Close region once and for all - r.close(); + region.close(); log.closeAndDelete(); } finally { - if(cluster != null) { - cluster.shutdown(); - } + StaticTestEnvironment.shutdownDfs(cluster); } } } diff --git a/src/test/org/apache/hadoop/hbase/TestHLog.java b/src/test/org/apache/hadoop/hbase/TestHLog.java index 63a333ab557..0366e5d29a5 100644 --- a/src/test/org/apache/hadoop/hbase/TestHLog.java +++ b/src/test/org/apache/hadoop/hbase/TestHLog.java @@ -49,13 +49,13 @@ public class TestHLog extends HBaseTestCase implements HConstants { try { // Write columns named 1, 2, 3, etc. and then values of single byte // 1, 2, 3... - TreeMap cols = new TreeMap(); + long timestamp = System.currentTimeMillis(); + TreeMap cols = new TreeMap(); for (int i = 0; i < COL_COUNT; i++) { - cols.put(new Text(Integer.toString(i)), + cols.put(new HStoreKey(row, new Text(Integer.toString(i)), timestamp), new byte[] { (byte)(i + '0') }); } - long timestamp = System.currentTimeMillis(); - log.append(regionName, tableName, row, cols, timestamp); + log.append(regionName, tableName, cols); long logSeqId = log.startCacheFlush(); log.completeCacheFlush(regionName, tableName, logSeqId); log.close(); diff --git a/src/test/org/apache/hadoop/hbase/TestHMemcache.java b/src/test/org/apache/hadoop/hbase/TestHMemcache.java index 68c8338ef81..d2a6ba0d95d 100644 --- a/src/test/org/apache/hadoop/hbase/TestHMemcache.java +++ b/src/test/org/apache/hadoop/hbase/TestHMemcache.java @@ -22,23 +22,16 @@ package org.apache.hadoop.hbase; import java.io.IOException; import java.io.UnsupportedEncodingException; import java.util.Map; -import java.util.SortedMap; import java.util.TreeMap; import junit.framework.TestCase; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hbase.HMemcache.Snapshot; import org.apache.hadoop.io.Text; /** memcache test case */ public class TestHMemcache extends TestCase { - private HMemcache hmemcache; - - private Configuration conf; + private HStore.Memcache hmemcache; private static final int ROW_COUNT = 3; @@ -50,10 +43,7 @@ public class TestHMemcache extends TestCase { @Override public void setUp() throws Exception { super.setUp(); - this.hmemcache = new HMemcache(); - // Set up a configuration that has configuration for a file - // filesystem implementation. - this.conf = new HBaseConfiguration(); + this.hmemcache = new HStore.Memcache(); } private Text getRowName(final int index) { @@ -69,48 +59,26 @@ public class TestHMemcache extends TestCase { * Adds {@link #ROW_COUNT} rows and {@link #COLUMNS_COUNT} * @param hmc Instance to add rows to. */ - private void addRows(final HMemcache hmc) throws UnsupportedEncodingException { + private void addRows(final HStore.Memcache hmc) + throws UnsupportedEncodingException { + for (int i = 0; i < ROW_COUNT; i++) { - TreeMap columns = new TreeMap(); + long timestamp = System.currentTimeMillis(); for (int ii = 0; ii < COLUMNS_COUNT; ii++) { Text k = getColumnName(i, ii); - columns.put(k, k.toString().getBytes(HConstants.UTF8_ENCODING)); + hmc.add(new HStoreKey(getRowName(i), k, timestamp), + k.toString().getBytes(HConstants.UTF8_ENCODING)); } - hmc.add(getRowName(i), columns, System.currentTimeMillis()); } } - private HLog getLogfile() throws IOException { - // Create a log file. - Path testDir = new Path(conf.get("hadoop.tmp.dir", - System.getProperty("java.tmp.dir")), "hbase"); - Path logFile = new Path(testDir, this.getName()); - FileSystem fs = testDir.getFileSystem(conf); - // Cleanup any old log file. - if (fs.exists(logFile)) { - fs.delete(logFile); - } - return new HLog(fs, logFile, this.conf); - } - - private Snapshot runSnapshot(final HMemcache hmc, final HLog log) - throws IOException { - + private void runSnapshot(final HStore.Memcache hmc) { // Save off old state. - int oldHistorySize = hmc.history.size(); - SortedMap oldMemcache = hmc.memcache; - // Run snapshot. - Snapshot s = hmc.snapshotMemcacheForLog(log); + int oldHistorySize = hmc.snapshot.size(); + hmc.snapshot(); // Make some assertions about what just happened. - assertEquals("Snapshot equals old memcache", hmc.snapshot, - oldMemcache); - assertEquals("Returned snapshot holds old memcache", - s.memcacheSnapshot, oldMemcache); - assertEquals("History has been incremented", - oldHistorySize + 1, hmc.history.size()); - assertEquals("History holds old snapshot", - hmc.history.get(oldHistorySize), oldMemcache); - return s; + assertTrue("History size has not increased", + oldHistorySize < hmc.snapshot.size()); } /** @@ -119,21 +87,14 @@ public class TestHMemcache extends TestCase { */ public void testSnapshotting() throws IOException { final int snapshotCount = 5; - final Text tableName = new Text(getName()); - HLog log = getLogfile(); // Add some rows, run a snapshot. Do it a few times. for (int i = 0; i < snapshotCount; i++) { addRows(this.hmemcache); - int historyInitialSize = this.hmemcache.history.size(); - Snapshot s = runSnapshot(this.hmemcache, log); - log.completeCacheFlush(new Text(Integer.toString(i)), - tableName, s.sequenceId); - // Clean up snapshot now we are done with it. - this.hmemcache.deleteSnapshot(); - assertTrue("History not being cleared", - historyInitialSize == this.hmemcache.history.size()); + runSnapshot(this.hmemcache); + this.hmemcache.getSnapshot(); + assertEquals("History not being cleared", 0, + this.hmemcache.snapshot.size()); } - log.closeAndDelete(); } private void isExpectedRow(final int rowIndex, TreeMap row) @@ -161,7 +122,8 @@ public class TestHMemcache extends TestCase { addRows(this.hmemcache); for (int i = 0; i < ROW_COUNT; i++) { HStoreKey hsk = new HStoreKey(getRowName(i)); - TreeMap all = this.hmemcache.getFull(hsk); + TreeMap all = new TreeMap(); + this.hmemcache.getFull(hsk, all); isExpectedRow(i, all); } } diff --git a/src/test/org/apache/hadoop/hbase/TestHRegion.java b/src/test/org/apache/hadoop/hbase/TestHRegion.java index 26df5f89d90..7b686f721d1 100644 --- a/src/test/org/apache/hadoop/hbase/TestHRegion.java +++ b/src/test/org/apache/hadoop/hbase/TestHRegion.java @@ -61,9 +61,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe read(); cleanup(); } finally { - if(cluster != null) { - cluster.shutdown(); - } + StaticTestEnvironment.shutdownDfs(cluster); } } @@ -78,14 +76,15 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe private static final Text CONTENTS_BODY = new Text("contents:body"); private static final Text CONTENTS_FIRSTCOL = new Text("contents:firstcol"); private static final Text ANCHOR_SECONDCOL = new Text("anchor:secondcol"); - + private MiniDFSCluster cluster = null; private FileSystem fs = null; private Path parentdir = null; private Path newlogdir = null; private HLog log = null; private HTableDescriptor desc = null; - HRegion region = null; + HRegion r = null; + HRegionIncommon region = null; private static int numInserted = 0; @@ -103,8 +102,9 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe desc = new HTableDescriptor("test"); desc.addFamily(new HColumnDescriptor("contents:")); desc.addFamily(new HColumnDescriptor("anchor:")); - region = new HRegion(parentdir, log, fs, conf, + r = new HRegion(parentdir, log, fs, conf, new HRegionInfo(desc, null, null), null); + region = new HRegionIncommon(r); } // Test basic functionality. Writes to contents:basic and anchor:anchornum-* @@ -129,7 +129,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - region.flushcache(false); + region.flushcache(); System.out.println("Cache flush elapsed time: " + ((System.currentTimeMillis() - startTime) / 1000.0)); @@ -169,7 +169,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe try { region.put(-1, CONTENTS_BASIC, "bad input".getBytes(HConstants.UTF8_ENCODING)); - } catch (LockException e) { + } catch (Exception e) { exceptionThrown = true; } assertTrue("Bad lock id", exceptionThrown); @@ -182,6 +182,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe String unregisteredColName = "FamilyGroup:FamilyLabel"; region.put(lockid, new Text(unregisteredColName), unregisteredColName.getBytes(HConstants.UTF8_ENCODING)); + region.commit(lockid); } catch (IOException e) { exceptionThrown = true; } finally { @@ -209,8 +210,8 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe for (int i = 0; i < lockCount; i++) { try { Text rowid = new Text(Integer.toString(i)); - lockids[i] = region.obtainRowLock(rowid); - rowid.equals(region.getRowFromLock(lockids[i])); + lockids[i] = r.obtainRowLock(rowid); + rowid.equals(r.getRowFromLock(lockids[i])); LOG.debug(getName() + " locked " + rowid.toString()); } catch (IOException e) { e.printStackTrace(); @@ -221,13 +222,8 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe // Abort outstanding locks. for (int i = lockCount - 1; i >= 0; i--) { - try { - region.abort(lockids[i]); - LOG.debug(getName() + " unlocked " + - Integer.toString(i)); - } catch (IOException e) { - e.printStackTrace(); - } + r.releaseRowLock(r.getRowFromLock(lockids[i])); + LOG.debug(getName() + " unlocked " + i); } LOG.debug(getName() + " released " + Integer.toString(lockCount) + " locks"); @@ -288,7 +284,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); HInternalScannerInterface s = - region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + r.getScanner(cols, new Text(), System.currentTimeMillis(), null); int numFetched = 0; try { HStoreKey curKey = new HStoreKey(); @@ -326,7 +322,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - region.flushcache(false); + region.flushcache(); System.out.println("Cache flush elapsed time: " + ((System.currentTimeMillis() - startTime) / 1000.0)); @@ -335,7 +331,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); @@ -390,7 +386,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); @@ -428,7 +424,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - region.flushcache(false); + region.flushcache(); System.out.println("Cache flush elapsed time: " + ((System.currentTimeMillis() - startTime) / 1000.0)); @@ -437,7 +433,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); numFetched = 0; try { HStoreKey curKey = new HStoreKey(); @@ -473,7 +469,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text("row_vals1_500"), + s = r.getScanner(cols, new Text("row_vals1_500"), System.currentTimeMillis(), null); numFetched = 0; @@ -542,7 +538,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe System.out.println("Flushing write #" + k); long flushStart = System.currentTimeMillis(); - region.flushcache(false); + region.flushcache(); long flushEnd = System.currentTimeMillis(); totalFlush += (flushEnd - flushStart); @@ -557,7 +553,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe } } long startCompact = System.currentTimeMillis(); - if(region.compactStores()) { + if(r.compactIfNeeded()) { totalCompact = System.currentTimeMillis() - startCompact; System.out.println("Region compacted - elapsedTime: " + (totalCompact / 1000.0)); @@ -583,43 +579,28 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe // NOTE: This test depends on testBatchWrite succeeding private void splitAndMerge() throws IOException { - Text midKey = new Text(); - - if(region.needsSplit(midKey)) { - System.out.println("Needs split"); - } - - // Split it anyway - - Text midkey = new Text("row_" - + (StaticTestEnvironment.debugging ? (N_ROWS / 2) : (NUM_VALS/2))); - - Path oldRegionPath = region.getRegionDir(); - + Path oldRegionPath = r.getRegionDir(); long startTime = System.currentTimeMillis(); + HRegion subregions[] = r.splitRegion(this); + if (subregions != null) { + System.out.println("Split region elapsed time: " + + ((System.currentTimeMillis() - startTime) / 1000.0)); - HRegion subregions[] = region.closeAndSplit(midkey, this); + assertEquals("Number of subregions", subregions.length, 2); - System.out.println("Split region elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); + // Now merge it back together - assertEquals("Number of subregions", subregions.length, 2); - - // Now merge it back together - - Path oldRegion1 = subregions[0].getRegionDir(); - Path oldRegion2 = subregions[1].getRegionDir(); - - startTime = System.currentTimeMillis(); - - region = HRegion.closeAndMerge(subregions[0], subregions[1]); - - System.out.println("Merge regions elapsed time: " - + ((System.currentTimeMillis() - startTime) / 1000.0)); - - fs.delete(oldRegionPath); - fs.delete(oldRegion1); - fs.delete(oldRegion2); + Path oldRegion1 = subregions[0].getRegionDir(); + Path oldRegion2 = subregions[1].getRegionDir(); + startTime = System.currentTimeMillis(); + r = HRegion.closeAndMerge(subregions[0], subregions[1]); + region = new HRegionIncommon(r); + System.out.println("Merge regions elapsed time: " + + ((System.currentTimeMillis() - startTime) / 1000.0)); + fs.delete(oldRegion1); + fs.delete(oldRegion2); + fs.delete(oldRegionPath); + } } /** @@ -650,7 +631,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe long startTime = System.currentTimeMillis(); HInternalScannerInterface s = - region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + r.getScanner(cols, new Text(), System.currentTimeMillis(), null); try { @@ -706,7 +687,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); try { int numFetched = 0; HStoreKey curKey = new HStoreKey(); @@ -744,7 +725,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe if(StaticTestEnvironment.debugging) { startTime = System.currentTimeMillis(); - s = region.getScanner(new Text[] { CONTENTS_BODY }, new Text(), + s = r.getScanner(new Text[] { CONTENTS_BODY }, new Text(), System.currentTimeMillis(), null); try { @@ -782,7 +763,7 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe startTime = System.currentTimeMillis(); - s = region.getScanner(cols, new Text(), System.currentTimeMillis(), null); + s = r.getScanner(cols, new Text(), System.currentTimeMillis(), null); try { int fetched = 0; @@ -817,22 +798,11 @@ public class TestHRegion extends HBaseTestCase implements RegionUnavailableListe } private void cleanup() { - - // Shut down the mini cluster try { log.closeAndDelete(); } catch (IOException e) { e.printStackTrace(); } - if (cluster != null) { - try { - fs.close(); - } catch (IOException e) { - e.printStackTrace(); - } - cluster.shutdown(); - cluster = null; - } // Delete all the DFS files diff --git a/src/test/org/apache/hadoop/hbase/TestHStoreFile.java b/src/test/org/apache/hadoop/hbase/TestHStoreFile.java index 64294604858..4db88e1754a 100644 --- a/src/test/org/apache/hadoop/hbase/TestHStoreFile.java +++ b/src/test/org/apache/hadoop/hbase/TestHStoreFile.java @@ -45,9 +45,13 @@ public class TestHStoreFile extends HBaseTestCase { /** {@inheritDoc} */ @Override public void setUp() throws Exception { - this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null); - this.fs = cluster.getFileSystem(); - this.dir = new Path(DIR, getName()); + try { + this.cluster = new MiniDFSCluster(this.conf, 2, true, (String[])null); + this.fs = cluster.getFileSystem(); + this.dir = new Path(DIR, getName()); + } catch (IOException e) { + StaticTestEnvironment.shutdownDfs(cluster); + } super.setUp(); } @@ -55,13 +59,7 @@ public class TestHStoreFile extends HBaseTestCase { @Override public void tearDown() throws Exception { super.tearDown(); - if (this.cluster != null) { - try { - this.cluster.shutdown(); - } catch (Exception e) { - LOG.warn("Closing down mini DFS", e); - } - } + StaticTestEnvironment.shutdownDfs(cluster); // ReflectionUtils.printThreadInfo(new PrintWriter(System.out), // "Temporary end-of-test thread dump debugging HADOOP-2040: " + getName()); } diff --git a/src/test/org/apache/hadoop/hbase/TestLogRolling.java b/src/test/org/apache/hadoop/hbase/TestLogRolling.java index 12705a26b59..4f3dc7f1b73 100644 --- a/src/test/org/apache/hadoop/hbase/TestLogRolling.java +++ b/src/test/org/apache/hadoop/hbase/TestLogRolling.java @@ -22,7 +22,6 @@ package org.apache.hadoop.hbase; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.dfs.MiniDFSCluster; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; @@ -90,6 +89,7 @@ public class TestLogRolling extends HBaseTestCase { super.setUp(); dfs = new MiniDFSCluster(conf, 2, true, (String[]) null); } catch (Exception e) { + StaticTestEnvironment.shutdownDfs(dfs); LOG.fatal("error during setUp: ", e); throw e; } @@ -100,21 +100,10 @@ public class TestLogRolling extends HBaseTestCase { public void tearDown() throws Exception { try { super.tearDown(); - if (cluster != null) { // shutdown mini HBase cluster cluster.shutdown(); } - - if (dfs != null) { - FileSystem fs = dfs.getFileSystem(); - try { - dfs.shutdown(); - } finally { - if (fs != null) { - fs.close(); - } - } - } + StaticTestEnvironment.shutdownDfs(dfs); } catch (Exception e) { LOG.fatal("error in tearDown", e); throw e; diff --git a/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java b/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java index a0c81a51f24..01ef6143da3 100644 --- a/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java +++ b/src/test/org/apache/hadoop/hbase/TestMasterAdmin.java @@ -36,6 +36,10 @@ public class TestMasterAdmin extends HBaseClusterTestCase { public TestMasterAdmin() { super(true); admin = null; + + // Make the thread wake frequency a little slower so other threads + // can run + conf.setInt("hbase.server.thread.wakefrequency", 2000); } /** @throws Exception */ diff --git a/src/test/org/apache/hadoop/hbase/TestScanner.java b/src/test/org/apache/hadoop/hbase/TestScanner.java index 0fce6de5bcf..c3a19edbaf5 100644 --- a/src/test/org/apache/hadoop/hbase/TestScanner.java +++ b/src/test/org/apache/hadoop/hbase/TestScanner.java @@ -50,7 +50,8 @@ public class TestScanner extends HBaseTestCase { private static final long START_CODE = Long.MAX_VALUE; - private HRegion region; + private HRegion r; + private HRegionIncommon region; /** Compare the HRegionInfo we read from HBase to what we stored */ private void validateRegionInfo(byte [] regionBytes) throws IOException { @@ -79,7 +80,7 @@ public class TestScanner extends HBaseTestCase { for(int i = 0; i < scanColumns.length; i++) { try { - scanner = region.getScanner(scanColumns[i], FIRST_ROW, + scanner = r.getScanner(scanColumns[i], FIRST_ROW, System.currentTimeMillis(), null); while(scanner.next(key, results)) { @@ -145,7 +146,8 @@ public class TestScanner extends HBaseTestCase { HLog log = new HLog(fs, new Path(regionDir, "log"), conf); - region = new HRegion(dir, log, fs, conf, REGION_INFO, null); + r = new HRegion(dir, log, fs, conf, REGION_INFO, null); + region = new HRegionIncommon(r); // Write information to the meta table @@ -165,9 +167,10 @@ public class TestScanner extends HBaseTestCase { // Close and re-open - region.close(); + r.close(); log.rollWriter(); - region = new HRegion(dir, log, fs, conf, REGION_INFO, null); + r = new HRegion(dir, log, fs, conf, REGION_INFO, null); + region = new HRegionIncommon(r); // Verify we can get the data back now that it is on disk. @@ -196,7 +199,7 @@ public class TestScanner extends HBaseTestCase { // flush cache - region.flushcache(false); + region.flushcache(); // Validate again @@ -205,9 +208,10 @@ public class TestScanner extends HBaseTestCase { // Close and reopen - region.close(); + r.close(); log.rollWriter(); - region = new HRegion(dir, log, fs, conf, REGION_INFO, null); + r = new HRegion(dir, log, fs, conf, REGION_INFO, null); + region = new HRegionIncommon(r); // Validate again @@ -232,7 +236,7 @@ public class TestScanner extends HBaseTestCase { // flush cache - region.flushcache(false); + region.flushcache(); // Validate again @@ -241,9 +245,10 @@ public class TestScanner extends HBaseTestCase { // Close and reopen - region.close(); + r.close(); log.rollWriter(); - region = new HRegion(dir, log, fs, conf, REGION_INFO, null); + r = new HRegion(dir, log, fs, conf, REGION_INFO, null); + region = new HRegionIncommon(r); // Validate again @@ -252,13 +257,11 @@ public class TestScanner extends HBaseTestCase { // clean up - region.close(); + r.close(); log.closeAndDelete(); } finally { - if(cluster != null) { - cluster.shutdown(); - } + StaticTestEnvironment.shutdownDfs(cluster); } } } diff --git a/src/test/org/apache/hadoop/hbase/TestScanner2.java b/src/test/org/apache/hadoop/hbase/TestScanner2.java index 3ca3cd06594..8b9ae91bbcf 100644 --- a/src/test/org/apache/hadoop/hbase/TestScanner2.java +++ b/src/test/org/apache/hadoop/hbase/TestScanner2.java @@ -71,6 +71,9 @@ public class TestScanner2 extends HBaseClusterTestCase { } } + /** + * @throws Exception + */ public void testStopRow() throws Exception { Text tableName = new Text(getName()); createTable(new HBaseAdmin(this.conf), tableName); @@ -86,6 +89,9 @@ public class TestScanner2 extends HBaseClusterTestCase { } } + /** + * @throws Exception + */ public void testIterator() throws Exception { HTable table = new HTable(this.conf, HConstants.ROOT_TABLE_NAME); HScannerInterface scanner = @@ -139,11 +145,12 @@ public class TestScanner2 extends HBaseClusterTestCase { int count = 0; while (scanner.next(key, results)) { for (Text c: results.keySet()) { + System.out.println(c); assertTrue(c.toString().matches(regexColumnname)); count++; } } - assertTrue(count == 1); + assertEquals(1, count); scanner.close(); } @@ -267,7 +274,7 @@ public class TestScanner2 extends HBaseClusterTestCase { Text tableName = new Text(getName()); admin.createTable(new HTableDescriptor(tableName.toString())); List regions = scan(metaTable); - assertEquals("Expected one region", regions.size(), 1); + assertEquals("Expected one region", 1, regions.size()); HRegionInfo region = regions.get(0); assertTrue("Expected region named for test", region.getRegionName().toString().startsWith(getName())); @@ -365,7 +372,7 @@ public class TestScanner2 extends HBaseClusterTestCase { // Assert added. byte [] bytes = t.get(region.getRegionName(), HConstants.COL_REGIONINFO); HRegionInfo hri = Writables.getHRegionInfo(bytes); - assertEquals(hri.getRegionId(), region.getRegionId()); + assertEquals(region.getRegionId(), hri.getRegionId()); if (LOG.isDebugEnabled()) { LOG.info("Added region " + region.getRegionName() + " to table " + t.getTableName()); diff --git a/src/test/org/apache/hadoop/hbase/TestSplit.java b/src/test/org/apache/hadoop/hbase/TestSplit.java index 38d6eba622b..6993868f11b 100644 --- a/src/test/org/apache/hadoop/hbase/TestSplit.java +++ b/src/test/org/apache/hadoop/hbase/TestSplit.java @@ -39,6 +39,9 @@ public class TestSplit extends MultiRegionTable { /** constructor */ public TestSplit() { super(); + + // Always compact if there is more than one store file. + conf.setInt("hbase.hstore.compactionThreshold", 2); // Make lease timeout longer, lease checks less frequent conf.setInt("hbase.master.lease.period", 10 * 1000); @@ -47,20 +50,15 @@ public class TestSplit extends MultiRegionTable { // Increase the amount of time between client retries conf.setLong("hbase.client.pause", 15 * 1000); + // This size should make it so we always split using the addContent + // below. After adding all data, the first region is 1.3M + conf.setLong("hbase.hregion.max.filesize", 1024 * 128); + Logger.getRootLogger().setLevel(Level.WARN); Logger.getLogger(this.getClass().getPackage().getName()). setLevel(Level.DEBUG); } - /** {@inheritDoc} */ - @Override - public void setUp() throws Exception { - super.setUp(); - // This size should make it so we always split using the addContent - // below. After adding all data, the first region is 1.3M - conf.setLong("hbase.hregion.max.filesize", 1024 * 128); - } - /** * Splits twice and verifies getting from each of the split regions. * @throws Exception @@ -83,7 +81,7 @@ public class TestSplit extends MultiRegionTable { private void basicSplit(final HRegion region) throws Exception { addContent(region, COLFAMILY_NAME3); - region.internalFlushcache(); + region.internalFlushcache(region.snapshotMemcaches()); Text midkey = new Text(); assertTrue(region.needsSplit(midkey)); HRegion [] regions = split(region); @@ -110,7 +108,12 @@ public class TestSplit extends MultiRegionTable { } addContent(regions[i], COLFAMILY_NAME2); addContent(regions[i], COLFAMILY_NAME1); - regions[i].internalFlushcache(); + long startTime = region.snapshotMemcaches(); + if (startTime == -1) { + LOG.info("cache flush not needed"); + } else { + regions[i].internalFlushcache(startTime); + } } // Assert that even if one store file is larger than a reference, the @@ -126,7 +129,7 @@ public class TestSplit extends MultiRegionTable { // To make regions splitable force compaction. for (int i = 0; i < regions.length; i++) { - assertTrue(regions[i].compactStores()); + regions[i].compactStores(); } TreeMap sortedMap = new TreeMap(); @@ -156,6 +159,12 @@ public class TestSplit extends MultiRegionTable { * @throws Exception */ public void testSplitRegionIsDeleted() throws Exception { + // Make sure the cache gets flushed so we trigger a compaction(s) and + // hence splits. This is done here rather than in the constructor because + // the first test runs without a cluster, and will block when the cache + // fills up. + conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024); + try { // Start up a hbase cluster MiniHBaseCluster cluster = new MiniHBaseCluster(conf, 1, true); @@ -228,7 +237,7 @@ public class TestSplit extends MultiRegionTable { assertTrue(r.needsSplit(midKey)); // Assert can get mid key from passed region. assertGet(r, COLFAMILY_NAME3, midKey); - HRegion [] regions = r.closeAndSplit(midKey, null); + HRegion [] regions = r.splitRegion(null); assertEquals(regions.length, 2); return regions; } diff --git a/src/test/org/apache/hadoop/hbase/TestTimestamp.java b/src/test/org/apache/hadoop/hbase/TestTimestamp.java index ef3916af36e..7fafc36f232 100644 --- a/src/test/org/apache/hadoop/hbase/TestTimestamp.java +++ b/src/test/org/apache/hadoop/hbase/TestTimestamp.java @@ -38,11 +38,11 @@ public class TestTimestamp extends HBaseTestCase { private static final String COLUMN_NAME = "contents:"; private static final Text COLUMN = new Text(COLUMN_NAME); - private static final Text[] COLUMNS = {COLUMN}; private static final Text ROW = new Text("row"); // When creating column descriptor, how many versions of a cell to allow. private static final int VERSIONS = 3; + /** * Test that delete works according to description in 1); } catch (Exception e) { - if (dfsCluster != null) { - dfsCluster.shutdown(); - dfsCluster = null; - } + StaticTestEnvironment.shutdownDfs(dfsCluster); throw e; } } + /** {@inheritDoc} */ @Override public void tearDown() throws Exception { super.tearDown(); @@ -127,9 +132,7 @@ public class TestTableIndex extends HBaseTestCase { hCluster.shutdown(); } - if (dfsCluster != null) { - dfsCluster.shutdown(); - } + StaticTestEnvironment.shutdownDfs(dfsCluster); } /** diff --git a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java b/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java index 448d4262796..adc54e63d05 100644 --- a/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java +++ b/src/test/org/apache/hadoop/hbase/mapred/TestTableMapReduce.java @@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.HTable; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MultiRegionTable; +import org.apache.hadoop.hbase.StaticTestEnvironment; import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; @@ -86,6 +87,21 @@ public class TestTableMapReduce extends MultiRegionTable { public TestTableMapReduce() { super(); + // Make the thread wake frequency a little slower so other threads + // can run + conf.setInt("hbase.server.thread.wakefrequency", 2000); + + // Make sure the cache gets flushed so we trigger a compaction(s) and + // hence splits. + conf.setInt("hbase.hregion.memcache.flush.size", 1024 * 1024); + + // Always compact if there is more than one store file. + conf.setInt("hbase.hstore.compactionThreshold", 2); + + // This size should make it so we always split using the addContent + // below. After adding all data, the first region is 1.3M + conf.setLong("hbase.hregion.max.filesize", 256 * 1024); + // Make lease timeout longer, lease checks less frequent conf.setInt("hbase.master.lease.period", 10 * 1000); conf.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000); @@ -97,9 +113,6 @@ public class TestTableMapReduce extends MultiRegionTable { @Override public void setUp() throws Exception { super.setUp(); - // This size is picked so the table is split into two - // after addContent in testMultiRegionTableMapReduce. - conf.setLong("hbase.hregion.max.filesize", 256 * 1024); dfsCluster = new MiniDFSCluster(conf, 1, true, (String[])null); try { fs = dfsCluster.getFileSystem(); @@ -109,10 +122,7 @@ public class TestTableMapReduce extends MultiRegionTable { hCluster = new MiniHBaseCluster(conf, 1, dfsCluster); LOG.info("Master is at " + this.conf.get(HConstants.MASTER_ADDRESS)); } catch (Exception e) { - if (dfsCluster != null) { - dfsCluster.shutdown(); - dfsCluster = null; - } + StaticTestEnvironment.shutdownDfs(dfsCluster); throw e; } } @@ -126,18 +136,7 @@ public class TestTableMapReduce extends MultiRegionTable { if(hCluster != null) { hCluster.shutdown(); } - - if (dfsCluster != null) { - dfsCluster.shutdown(); - } - - if (fs != null) { - try { - fs.close(); - } catch (IOException e) { - LOG.info("During tear down got a " + e.getMessage()); - } - } + StaticTestEnvironment.shutdownDfs(dfsCluster); } /** @@ -218,49 +217,54 @@ public class TestTableMapReduce extends MultiRegionTable { // insert some data into the test table HTable table = new HTable(conf, new Text(SINGLE_REGION_TABLE_NAME)); - for(int i = 0; i < values.length; i++) { - long lockid = table.startUpdate(new Text("row_" - + String.format("%1$05d", i))); + try { + for(int i = 0; i < values.length; i++) { + long lockid = table.startUpdate(new Text("row_" + + String.format("%1$05d", i))); + + try { + table.put(lockid, TEXT_INPUT_COLUMN, values[i]); + table.commit(lockid, System.currentTimeMillis()); + lockid = -1; + } finally { + if (lockid != -1) + table.abort(lockid); + } + } + + LOG.info("Print table contents before map/reduce"); + scanTable(conf, SINGLE_REGION_TABLE_NAME); + + @SuppressWarnings("deprecation") + MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1); try { - table.put(lockid, TEXT_INPUT_COLUMN, values[i]); - table.commit(lockid, System.currentTimeMillis()); - lockid = -1; + JobConf jobConf = new JobConf(conf, TestTableMapReduce.class); + jobConf.setJobName("process column contents"); + jobConf.setNumMapTasks(1); + jobConf.setNumReduceTasks(1); + + TableMap.initJob(SINGLE_REGION_TABLE_NAME, INPUT_COLUMN, + ProcessContentsMapper.class, jobConf); + + TableReduce.initJob(SINGLE_REGION_TABLE_NAME, + IdentityTableReduce.class, jobConf); + + JobClient.runJob(jobConf); + } finally { - if (lockid != -1) - table.abort(lockid); + mrCluster.shutdown(); } - } - - LOG.info("Print table contents before map/reduce"); - scanTable(conf, SINGLE_REGION_TABLE_NAME); - @SuppressWarnings("deprecation") - MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1); + LOG.info("Print table contents after map/reduce"); + scanTable(conf, SINGLE_REGION_TABLE_NAME); - try { - JobConf jobConf = new JobConf(conf, TestTableMapReduce.class); - jobConf.setJobName("process column contents"); - jobConf.setNumMapTasks(1); - jobConf.setNumReduceTasks(1); + // verify map-reduce results + verify(conf, SINGLE_REGION_TABLE_NAME); - TableMap.initJob(SINGLE_REGION_TABLE_NAME, INPUT_COLUMN, - ProcessContentsMapper.class, jobConf); - - TableReduce.initJob(SINGLE_REGION_TABLE_NAME, - IdentityTableReduce.class, jobConf); - - JobClient.runJob(jobConf); - } finally { - mrCluster.shutdown(); + table.close(); } - - LOG.info("Print table contents after map/reduce"); - scanTable(conf, SINGLE_REGION_TABLE_NAME); - - // verify map-reduce results - verify(conf, SINGLE_REGION_TABLE_NAME); } /* @@ -277,37 +281,42 @@ public class TestTableMapReduce extends MultiRegionTable { admin.createTable(desc); // Populate a table into multiple regions - MultiRegionTable.makeMultiRegionTable(conf, hCluster, fs, - MULTI_REGION_TABLE_NAME, INPUT_COLUMN); + makeMultiRegionTable(conf, hCluster, fs, MULTI_REGION_TABLE_NAME, + INPUT_COLUMN); // Verify table indeed has multiple regions HTable table = new HTable(conf, new Text(MULTI_REGION_TABLE_NAME)); - Text[] startKeys = table.getStartKeys(); - assertTrue(startKeys.length > 1); - - @SuppressWarnings("deprecation") - MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1); - try { - JobConf jobConf = new JobConf(conf, TestTableMapReduce.class); - jobConf.setJobName("process column contents"); - jobConf.setNumMapTasks(2); - jobConf.setNumReduceTasks(1); + Text[] startKeys = table.getStartKeys(); + assertTrue(startKeys.length > 1); - TableMap.initJob(MULTI_REGION_TABLE_NAME, INPUT_COLUMN, - ProcessContentsMapper.class, jobConf); + @SuppressWarnings("deprecation") + MiniMRCluster mrCluster = new MiniMRCluster(2, fs.getUri().toString(), 1); - TableReduce.initJob(MULTI_REGION_TABLE_NAME, - IdentityTableReduce.class, jobConf); + try { + JobConf jobConf = new JobConf(conf, TestTableMapReduce.class); + jobConf.setJobName("process column contents"); + jobConf.setNumMapTasks(2); + jobConf.setNumReduceTasks(1); - JobClient.runJob(jobConf); + TableMap.initJob(MULTI_REGION_TABLE_NAME, INPUT_COLUMN, + ProcessContentsMapper.class, jobConf); + + TableReduce.initJob(MULTI_REGION_TABLE_NAME, + IdentityTableReduce.class, jobConf); + + JobClient.runJob(jobConf); + + } finally { + mrCluster.shutdown(); + } + + // verify map-reduce results + verify(conf, MULTI_REGION_TABLE_NAME); } finally { - mrCluster.shutdown(); + table.close(); } - - // verify map-reduce results - verify(conf, MULTI_REGION_TABLE_NAME); } private void scanTable(HBaseConfiguration conf, String tableName)